From 0b6431bbe33c5ea530d0dec6a2332909509a594d Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 12 Jan 2021 11:56:14 +0300 Subject: [PATCH 01/98] IGNITE-13805 (wip, draft) Restore local files ignore distrbution. --- .../org/apache/ignite/IgniteSnapshot.java | 3 + .../snapshot/IgniteSnapshotManager.java | 210 ++++++ .../SnapshotRestoreCacheGroupProcess.java | 566 +++++++++++++++ .../SnapshotRestorePerformResponse.java | 28 + .../SnapshotRestorePrepareResponse.java | 96 +++ .../snapshot/SnapshotRestoreRequest.java | 88 +++ .../SnapshotRestoreRollbackResponse.java | 28 + .../cluster/GridClusterStateProcessor.java | 6 + .../util/distributed/DistributedProcess.java | 8 +- .../snapshot/AbstractSnapshotSelfTest.java | 6 +- .../IgniteClusterSnapshoRestoreSelfTest.java | 679 ++++++++++++++++++ .../IgnitePdsWithIndexingTestSuite.java | 4 +- 12 files changed, 1717 insertions(+), 5 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index 3623c1f8e66d18..a5f3110415d21c 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -17,6 +17,7 @@ package org.apache.ignite; +import java.util.Collection; import org.apache.ignite.lang.IgniteFuture; /** @@ -48,4 +49,6 @@ public interface IgniteSnapshot { * @return Future which will be completed when cancel operation finished. */ public IgniteFuture cancelSnapshot(String name); + + public IgniteFuture restoreCacheGroups(String snapshotName, Collection cacheGroupNames); } 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 d23b5843579f94..f456c8c9c63768 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 @@ -53,6 +53,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; import java.util.function.Function; @@ -72,6 +73,8 @@ import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; +import org.apache.ignite.internal.binary.BinaryMetadata; +import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener; @@ -82,6 +85,7 @@ import org.apache.ignite.internal.processors.cache.CacheType; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; @@ -160,6 +164,9 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.pageIndex; import static org.apache.ignite.internal.pagemem.PageIdUtils.toDetailString; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.resolveBinaryWorkDir; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE; @@ -300,6 +307,8 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Last seen cluster snapshot operation. */ private volatile ClusterSnapshotFuture lastSeenSnpFut = new ClusterSnapshotFuture(); + private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProcess; + /** * @param ctx Kernal context. */ @@ -315,6 +324,8 @@ public IgniteSnapshotManager(GridKernalContext ctx) { this::processLocalSnapshotEndStageResult); marsh = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName()); + + restoreCacheGrpProcess = new SnapshotRestoreCacheGroupProcess(ctx); } /** @@ -410,6 +421,8 @@ public static String partDeltaFileName(int partId) { "One of baseline nodes left the cluster: " + leftNodeId)); } } + + restoreCacheGrpProcess.onNodeLeft(leftNodeId); } } finally { @@ -747,6 +760,10 @@ public boolean isSnapshotCreating() { } } + public boolean isSnapshotRestoring() { + return restoreCacheGrpProcess.inProgress(); + } + /** * @return List of all known snapshots on the local node. */ @@ -1061,6 +1078,171 @@ public List readSnapshotMetadatas(String snpName) { } } + /** {@inheritDoc} */ + @Override public IgniteFuture restoreCacheGroups(String snpName, Collection grpNames) { + return restoreCacheGrpProcess.start(snpName, grpNames); + } + + protected void ensureMetaCanBeMerged(String snpName) throws IgniteCheckedException, IOException { + String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); + + File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); + + String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + + nodeFolderName; + + File snapshotMetadataDir = new File(workDIr, subPath); + + if (!snapshotMetadataDir.exists()) + return; + + // todo get binaryContext without cast + CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); + + for (File file : snapshotMetadataDir.listFiles()) { + try (FileInputStream in = new FileInputStream(file)) { + BinaryMetadata meta = U.unmarshal(cctx.kernalContext().config().getMarshaller(), in, U.resolveClassLoader(cctx.kernalContext().config())); + + BinaryMetadata oldMeta = binProc.metadata0(meta.typeId()); + + if (oldMeta == null) + continue; + + BinaryUtils.mergeMetadata(oldMeta, meta, null); + } + } + } + + protected RestoreOperationContext restoreCacheGroupsLocal(String snpName, Collection grpNames) throws IgniteCheckedException { + RestoreOperationContext opCtx = new RestoreOperationContext(); + + String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); + + File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); + + String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + + nodeFolderName; + + File snapshotMetadataDir = new File(workDIr, subPath); + + if (!snapshotMetadataDir.exists()) + return opCtx; + + // restore metadata + CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); + + List metas = new ArrayList<>(); + + Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); + ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); + + for (File file : snapshotMetadataDir.listFiles()) { + try (FileInputStream in = new FileInputStream(file)) { + metas.add(U.unmarshal(marshaller, in, clsLdr)); + } + catch (Exception e) { + throw new IgniteCheckedException("Failed to add metadata from file: " + file.getName() + + "; exception was thrown: " + e.getMessage()); + } + } + + // todo should register only from one node and validate result + if (!F.isEmpty(metas)) { + Future updateMetaFut = cctx.kernalContext().getSystemExecutorService().submit(() -> { + for (BinaryMetadata meta : metas) + procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); + }); + + opCtx.updateMetaFuture(updateMetaFut); + } + + for (String grpName : grpNames) { + File cacheDir = resolveCacheDir(grpName); + + assert F.isEmpty(cacheDir.list()) : cacheDir; + + File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); + + if (!snapshotCacheDir.exists()) { + log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); + + continue; + } + + if (!cacheDir.exists()) { + cacheDir.mkdir(); + + opCtx.cacheGroupFile(grpName, cacheDir); + } + + try { + for (File snpFile : snapshotCacheDir.listFiles()) { + File target = new File(cacheDir, snpFile.getName()); + + log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); + + Files.copy(snpFile.toPath(), target.toPath()); + + opCtx.cacheGroupFile(grpName, target); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); + } + } + + return opCtx; + } + + protected void rollbackRestoreOperation(Collection grps, RestoreOperationContext opCtx) { + for (String grpName : grps) { + List files = opCtx.cacheGroupFiles(grpName); + + List dirs = new ArrayList<>(); + + for (File file : files) { + if (!file.exists()) + continue; + + if (file.isDirectory()) + dirs.add(file); + + file.delete(); + } + + for (File dir : dirs) + dir.delete(); + } + } + + private File resolveCacheDir(String cacheOrGrpName) throws IgniteCheckedException { + File workDIr = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); + + String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName() + File.separator; + + File cacheDir = new File(workDIr, nodeDirName + CACHE_DIR_PREFIX + cacheOrGrpName); + + if (cacheDir.exists()) + return cacheDir; + + return new File(workDIr, nodeDirName + CACHE_GRP_DIR_PREFIX + cacheOrGrpName); + } + + protected File resolveSnapshotCacheDir(String snpName, IgniteConfiguration cfg, String cacheName) throws IgniteCheckedException { + File workDIr = resolveSnapshotWorkDirectory(cfg); + + String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName() + File.separator; + + String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + nodeDirName + File.separator; + + File cacheDir = new File(workDIr, subPath + CACHE_DIR_PREFIX + cacheName); + + if (cacheDir.exists()) + return cacheDir; + + return new File(workDIr, subPath + CACHE_GRP_DIR_PREFIX + cacheName); + } + /** {@inheritDoc} */ @Override public void onReadyForReadWrite(ReadWriteMetastorage metaStorage) throws IgniteCheckedException { synchronized (snpOpMux) { @@ -1984,4 +2166,32 @@ public IgniteSnapshotFutureImpl(IgniteInternalFuture fut) { return new IgniteException("Snapshot has not been created", U.convertException(e)); } } + + static class RestoreOperationContext { + private final Map> files = new HashMap<>(); + + private final List metadataTypes = new ArrayList<>(); + + private volatile Future updateMetaFuture; + + public void updateMetaFuture(Future updateMetaFuture) { + this.updateMetaFuture = updateMetaFuture; + } + + public @Nullable Future updateMetaFuture() { + return this.updateMetaFuture; + } + + public void cacheGroupFile(String grpName, File newFile) { + files.computeIfAbsent(grpName, v -> new ArrayList<>()).add(newFile); + } + + public void metadataType(int typeId) { + metadataTypes.add(typeId); + } + + public List cacheGroupFiles(String grpName) { + return files.get(grpName); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java new file mode 100644 index 00000000000000..e2e427734bfcf4 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -0,0 +1,566 @@ +/* + * 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.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +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 java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; +import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.RestoreOperationContext; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestorePrepareResponse.CacheGroupSnapshotDetails; +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.future.IgniteFinishedFutureImpl; +import org.apache.ignite.internal.util.future.IgniteFutureImpl; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteBiTuple; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.marshaller.MarshallerUtils; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.jetbrains.annotations.Nullable; + +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.END_SNAPSHOT_RESTORE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT_RESTORE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.UNDO_SNAPSHOT_RESTORE; + +/** + * Distributed process to restore cache group from the snapshot. + */ +public class SnapshotRestoreCacheGroupProcess { + /** Reject operation message. */ + private static final String OP_REJECT_MSG = "Snapshot restore operation was rejected. "; + + /** Kernal context. */ + private final GridKernalContext ctx; + + /** Cache group restore prepare phase. */ + private final DistributedProcess prepareRestoreProc; + + /** Cache group restore perform phase. */ + private final DistributedProcess performRestoreProc; + + /** Cache group restore rollback phase. */ + private final DistributedProcess rollbackRestoreProc; + + /** Logger. */ + private final IgniteLogger log; + + /** The future to be completed when the cache restore process is complete. */ + private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); + + /** + * @param ctx Kernal context. + */ + public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { + this.ctx = ctx; + + log = ctx.log(getClass()); + + prepareRestoreProc = new DistributedProcess<>(ctx, START_SNAPSHOT_RESTORE, this::prepare, this::finishPrepare); + performRestoreProc = new DistributedProcess<>(ctx, END_SNAPSHOT_RESTORE, this::perform, this::finishPerform); + rollbackRestoreProc = new DistributedProcess<>(ctx, UNDO_SNAPSHOT_RESTORE, this::rollback, this::finishRollback); + + fut.onDone(); + } + + /** + * Start cache group restore operation. + * + * @param snpName Snapshot name. + * @param cacheGrpNames Name of the cache groups for restore. + * @return Future that will be completed when the restore operation is complete and the cache groups are started. + */ + public IgniteFuture start(String snpName, Collection cacheGrpNames) { + if (ctx.clientNode()) { + return new IgniteFinishedFutureImpl<>(new UnsupportedOperationException("Client and daemon nodes can not " + + "perform this operation.")); + } + + IgniteInternalFuture fut0 = fut; + + if (!fut0.isDone()) { + return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + + "The previous snapshot restore operation was not completed.")); + } + + if (!ctx.state().clusterState().state().active()) + return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); + + Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + F.node2id(), + (node) -> CU.baselineNode(node, ctx.state().clusterState()))); + + SnapshotRestoreRequest req = new SnapshotRestoreRequest(snpName, cacheGrpNames, srvNodeIds); + + fut = new RestoreSnapshotFuture(true); + + prepareRestoreProc.start(req.requestId(), req); + + return new IgniteFutureImpl<>(fut); + } + + /** + * Check if the cache group restore process is currently running. + * + * @return {@code True} if cache group restore process is currently running. + */ + public boolean inProgress() { + RestoreSnapshotFuture fut0 = fut; + + return !fut0.isDone() && fut0.request() != null; + } + + /** + * Node left callback. + * + * @param leftNodeId Left node ID. + */ + public void onNodeLeft(UUID leftNodeId) { + RestoreSnapshotFuture fut0 = fut; + + if (fut0.isDone()) + return; + + SnapshotRestoreRequest req = fut0.request(); + + if (req != null && req.requiredNodes().contains(leftNodeId)) { + fut.handleError(new IgniteException(OP_REJECT_MSG + + "Baseline node has left the cluster [nodeId=" + leftNodeId + ']')); + } + } + + private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { + if (ctx.clientNode()) + return new GridFinishedFuture<>(); + + if (inProgress()) + return errResponse(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); + + if (!ctx.state().clusterState().state().active()) + return errResponse(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); + + if (fut.isDone()) + fut = new RestoreSnapshotFuture(false); + + fut.request(req); + + List grpCfgs = new ArrayList<>(); + + // Collect cache configuration(s). + for (String cacheName : req.groups()) { + try { + CacheGroupSnapshotDetails grpCfg = readCacheGroupDetails(req.snapshotName(), cacheName); + + if (grpCfg != null) + grpCfgs.add(grpCfg); + } + catch (IOException | IgniteCheckedException e) { + return new GridFinishedFuture<>(e); + } + } + + if (grpCfgs.isEmpty()) + return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); + + try { + ctx.cache().context().snapshotMgr().ensureMetaCanBeMerged(req.snapshotName()); + } + catch (BinaryObjectException e) { + log.warning(OP_REJECT_MSG + "Incompatible binary types found", e); + + return errResponse(OP_REJECT_MSG + "Incompatible binary types found: " + e.getMessage()); + } + catch (IOException | IgniteCheckedException e) { + return errResponse(new IgniteException("Prepare phase has failed: " + e.getMessage(), e)); + } + + return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); + } + + private @Nullable CacheGroupSnapshotDetails readCacheGroupDetails(String snapshotName, String grpName) throws IgniteCheckedException, IOException { + File cacheDir = ctx.cache().context().snapshotMgr().resolveSnapshotCacheDir(snapshotName, ctx.config(), grpName); + + if (!cacheDir.exists()) + return null; + + Set parts = new HashSet<>(); + + List> cacheCfgs = new ArrayList<>(1); + + for (File file : cacheDir.listFiles()) { + if (file.isDirectory()) + continue; + + String name = file.getName(); + + if (name.endsWith(CACHE_DATA_FILENAME) && file.length() > 0) + cacheCfgs.add(unmarshal(ctx.config(), file)); + else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { + String partId = name.substring(FilePageStoreManager.PART_FILE_PREFIX.length(), name.indexOf('.')); + + parts.add(Integer.parseInt(partId)); + } + } + + return new CacheGroupSnapshotDetails(grpName, cacheCfgs, parts); + } + + private void finishPrepare(UUID reqId, Map res, Map errs) { + RestoreSnapshotFuture fut0 = fut; + + if (!errs.isEmpty()) { + completeFuture(reqId, errs, fut0); + + return; + } + + if (fut0.failure() != null) { + fut0.onDone(fut0.failure()); + + return; + } + + List notFoundGroups = new ArrayList<>(fut.request().groups()); + + try { + Collection grpsDetails = mergeDetails(res); + + List cacheCfgs = new ArrayList<>(); + + for (CacheGroupSnapshotDetails grpDetails : grpsDetails) { + CacheConfiguration ccfg = F.first(grpDetails.configs()); + + if (ccfg == null) + continue; + + int reqParts = ccfg.getAffinity().partitions(); + int availParts = grpDetails.parts().size(); + + if (reqParts != availParts) { + throw new IgniteCheckedException("Cannot restore snapshot, not all partitions available [" + + "required=" + reqParts + ", avail=" + availParts + ", grp=" + grpDetails.groupName() + ']'); + } + + notFoundGroups.remove(grpDetails.groupName()); + + cacheCfgs.addAll(grpDetails.configs()); + + CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(grpDetails.groupName())); + + if (desc != null) { + throw new IllegalStateException("Cache group \"" + desc.cacheOrGroupName() + + "\" should be destroyed manually before perform restore operation."); + } + } + + if (!notFoundGroups.isEmpty()) { + throw new IllegalArgumentException("Cache group(s) \"" + F.concat(notFoundGroups, ", ") + + "\" not found in snapshot \"" + fut.request().snapshotName() + "\""); + } + + Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + F.node2id(), + (node) -> CU.baselineNode(node, ctx.state().clusterState()))); + + Set reqNodes = new HashSet<>(fut.request().requiredNodes()); + + reqNodes.removeAll(srvNodeIds); + + if (!reqNodes.isEmpty()) { + throw new IllegalStateException("Unable to perform a restore operation, server node(s) left " + + "the cluster [nodeIds=" + F.concat(reqNodes, ", ") + ']'); + } + + fut.startConfigs(cacheCfgs); + } + catch (Exception e) { + fut.onDone(e); + + return; + } + + if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut.isDone()) + performRestoreProc.start(reqId, fut.request()); + } + + private Collection mergeDetails(Map responses) { + Map> globalDetails = new HashMap<>(); + + for (Map.Entry entry : responses.entrySet()) { + UUID currNodeId = entry.getKey(); + SnapshotRestorePrepareResponse singleResp = entry.getValue(); + + for (CacheGroupSnapshotDetails nodeDetails : singleResp.groups()) { + T2 clusterDetailsPair = globalDetails.get(nodeDetails.groupName()); + + String grpName = nodeDetails.groupName(); + + if (clusterDetailsPair == null) { + globalDetails.put(grpName, new T2<>(currNodeId, nodeDetails)); + + continue; + } + + CacheGroupSnapshotDetails clusterDetails = clusterDetailsPair.get2(); + + int currCfgCnt = nodeDetails.configs().size(); + int savedCfgCnt = clusterDetails.configs().size(); + + if (currCfgCnt != savedCfgCnt) { + throw new IllegalStateException("Count of cache configs in shared group mismatch [" + + "node1=" + clusterDetailsPair.get1() + ", cnt=" + savedCfgCnt + + ", node2=" + currNodeId + ", cnt=" + nodeDetails.configs().size() + ']'); + } + + clusterDetails.parts().addAll(nodeDetails.parts()); + } + } + + return F.viewReadOnly(globalDetails.values(), IgniteBiTuple::get2); + } + + private IgniteInternalFuture perform(SnapshotRestoreRequest req) { + if (ctx.clientNode() || !req.requiredNodes().contains(ctx.localNodeId())) + return new GridFinishedFuture<>(); + + if (!req.equals(fut.request())) + return errResponse("Unknown snapshot restore operation was rejected."); + + try { + RestoreOperationContext opCtx = + ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups()); + + fut.rollbackContext(opCtx); + + return new GridFinishedFuture<>(new SnapshotRestorePerformResponse()); + } catch (IgniteCheckedException e) { + RestoreSnapshotFuture fut0 = fut; + + if (fut0 != null && fut0.id().equals(req.requestId())) + fut0.onDone(e); + + return new GridFinishedFuture<>(e); + } + } + + private void finishPerform(UUID reqId, Map map, Map errs) { + RestoreSnapshotFuture fut0 = fut; + + if (!F.isEmpty(errs)) { + completeFuture(reqId, errs, fut0); + + return; + } + + Throwable failure = fut0.failure(); + + // todo not only left nodes + if (failure == null && !map.keySet().containsAll(fut0.request().requiredNodes())) { + Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); + + reqNodes.removeAll(map.keySet()); + + log.warning("Node left the cluster, snapshot restore operation should be reverted [nodeIds=" + F.concat(reqNodes, ", ")); + + fut0.handleError(failure = new IgniteException(new IgniteException(OP_REJECT_MSG + + "Baseline node has left the cluster [nodeId(s)=" + F.concat(reqNodes, ", ") + ']'))); + } + + if (failure != null) { + if (U.isLocalNodeCoordinator(ctx.discovery())) { + log.info("Starting rollback routine."); + + rollbackRestoreProc.start(reqId, fut0.request()); + } + + return; + } + + Collection ccfgs0 = fut0.startConfigs(); + + if (fut0 == null || !fut0.id().equals(reqId) || !fut0.initiator() || F.isEmpty(ccfgs0)) { + completeFuture(reqId, errs, fut); + + return; + } + + ctx.cache().dynamicStartCaches(ccfgs0, true, true, false). + listen(f -> completeFuture(reqId, errs, fut0)); + } + + // todo separate rollback request + private IgniteInternalFuture rollback(SnapshotRestoreRequest req) { + if (ctx.clientNode()) + return new GridFinishedFuture<>(); + + RestoreSnapshotFuture fut0 = fut; + + // + if (!req.equals(fut.request())) + return errResponse("Unknown snapshot restore operation was rejected [fut=" + fut + ", req=" + req + ']'); + + if (fut0.rollbackContext() != null) + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(req.groups(), fut0.rollbackContext()); + + return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse()); + } + + private void finishRollback(UUID reqId, Map map, Map errs) { + RestoreSnapshotFuture fut0 = fut; + + if (!F.isEmpty(errs)) { + completeFuture(reqId, errs, fut0); + + return; + } + + fut0.onDone(fut0.failure()); + //if (!fut0.id().equals(reqId) || !fut0.initiator()) + } + + /** + * @param reqId Request id. + * @param err Exception. + * @param fut Key change future. + * @return {@code True} if future was completed by this call. + */ + private boolean completeFuture(UUID reqId, Map err, RestoreSnapshotFuture fut) { + if (!fut.id().equals(reqId) || fut.isDone()) + return false; + + return !F.isEmpty(err) ? fut.onDone(F.firstValue(err)) : fut.onDone(); + } + + private CacheConfiguration unmarshal(IgniteConfiguration cfg, File cacheDataFile) throws IOException, IgniteCheckedException { + JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(cfg.getIgniteInstanceName()); + + try (InputStream stream = new BufferedInputStream(new FileInputStream(cacheDataFile))) { + StoredCacheData data = marshaller.unmarshal(stream, U.resolveClassLoader(cfg)); + + return data.config(); + } + } + + private IgniteInternalFuture errResponse(String msg) { + return errResponse(new IgniteException(msg)); + } + + private IgniteInternalFuture errResponse(Exception ex) { + //return errResponse(msg, null); + return new GridFinishedFuture<>(ex); + } + + /** */ + protected static class RestoreSnapshotFuture extends GridFutureAdapter { + /** Request ID. */ + private final boolean initiator; + + private final AtomicReference reqRef = new AtomicReference<>(); + + private volatile RestoreOperationContext rollbackCtx; + + private volatile Throwable err; + + public Throwable failure() { + return err; + } + + private volatile Collection cacheCfgsToStart; + + public SnapshotRestoreRequest request() { + return reqRef.get(); + } + + public boolean request(SnapshotRestoreRequest req) { + return reqRef.compareAndSet(null, req); + } + + /** @param id Request ID. */ + RestoreSnapshotFuture(boolean initiator) { + this.initiator = initiator; + } + + public boolean initiator() { + return initiator; + } + + /** @return Request ID. */ + public UUID id() { + SnapshotRestoreRequest req = reqRef.get(); + + return req != null ? req.requestId() : null; + } + + public void handleError(Throwable err) { + this.err = err; + } + + public void startConfigs(Collection ccfgs) { + cacheCfgsToStart = ccfgs; + } + + public Collection startConfigs() { + return cacheCfgsToStart; + } + + public void rollbackContext(RestoreOperationContext opCtx) { + rollbackCtx = opCtx; + } + + public RestoreOperationContext rollbackContext() { + return rollbackCtx; + } + + @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { + return super.onDone(res, err, cancel); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(RestoreSnapshotFuture.class, this); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java new file mode 100644 index 00000000000000..4c4e5da4f5d7ed --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java @@ -0,0 +1,28 @@ +/* + * 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; + +/** + * Snapshot restore operation single node response. + */ +public class SnapshotRestorePerformResponse implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java new file mode 100644 index 00000000000000..5172ba4104d3b8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -0,0 +1,96 @@ +/* + * 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.List; +import java.util.Set; +import org.apache.ignite.configuration.CacheConfiguration; + +/** + * Snapshot restore operation single node validation response. + */ +public class SnapshotRestorePrepareResponse implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + private List groups; + + /** */ + public SnapshotRestorePrepareResponse() { + // No-op. + } + + /** + * @param groups List of cache groups snapshot details. + */ + public SnapshotRestorePrepareResponse(List groups) { + this.groups = groups; + } + + /** todo */ + public List groups() { + return groups; + } + + /** */ + static class CacheGroupSnapshotDetails implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Cache group name. */ + private String grpName; + + /** Local partition IDs. */ + private Set parts; + + /** Group cache configurations. */ + private List> cfgs; + + /** + * @param cfgs Group cache configurations. + * @param parts Local partition IDs. + */ + public CacheGroupSnapshotDetails(String grpName, List> cfgs, Set parts) { + this.grpName = grpName; + this.cfgs = cfgs; + this.parts = parts; + } + + /** + * @return Group cache configurations. + */ + public List> configs() { + return cfgs; + } + + /** + * @return Local partition IDs. + */ + public Set parts() { + return parts; + } + + /** + * @return Cache group name. + */ + public String groupName() { + return grpName; + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java new file mode 100644 index 00000000000000..61c15cad404f73 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java @@ -0,0 +1,88 @@ +/* + * 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.Objects; +import java.util.Set; +import java.util.UUID; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +public class SnapshotRestoreRequest implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Snapshot name. */ + private final String snpName; + + /** The list of cache groups to restore from the snapshot. */ + @GridToStringInclude + private final Collection grps; + + @GridToStringInclude + private final Set reqNodes; + + /** Request ID. */ + private final UUID reqId = UUID.randomUUID(); + + public SnapshotRestoreRequest(String snpName, Collection grps, Set reqNodes) { + this.snpName = snpName; + this.grps = grps; + this.reqNodes = reqNodes; + } + + public UUID requestId() { + return reqId; + } + + public Collection groups() { + return grps; + } + + public String snapshotName() { + return snpName; + } + + public Set requiredNodes() { + return Collections.unmodifiableSet(reqNodes); + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + return Objects.equals(reqId, ((SnapshotRestoreRequest)o).reqId); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(reqId); + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestoreRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java new file mode 100644 index 00000000000000..6153372308cba8 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java @@ -0,0 +1,28 @@ +/* + * 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; + +/** + * Snapshot restore operation single node response. + */ +public class SnapshotRestoreRollbackResponse implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index a774d520a20386..e39fa302f403e5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -1072,6 +1072,12 @@ public IgniteInternalFuture changeGlobalState( ); } +// if (state == INACTIVE && ctx.cache().context().snapshotMgr().isSnapshotRestoring()) { +// return new GridFinishedFuture<>( +// new IllegalStateException("The cluster cannot be deactivated until the snapshot restore operation is complete.") +// ); +// } + BaselineTopology blt = (compatibilityMode && !forceChangeBaselineTopology) ? null : calculateNewBaselineTopology(state, baselineNodes, forceChangeBaselineTopology); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index 7d9e6ae391482a..d82b48e96c6b6a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -445,6 +445,12 @@ public enum DistributedProcessType { /** * Rotate performance statistics. */ - PERFORMANCE_STATISTICS_ROTATE + PERFORMANCE_STATISTICS_ROTATE, + + START_SNAPSHOT_RESTORE, + + END_SNAPSHOT_RESTORE, + + UNDO_SNAPSHOT_RESTORE } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 74976b13efc881..137c3f5580cfdd 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -111,7 +111,7 @@ public abstract class AbstractSnapshotSelfTest extends GridCommonAbstractTest { protected final List locEvts = new CopyOnWriteArrayList<>(); /** Configuration for the 'default' cache. */ - protected volatile CacheConfiguration dfltCacheCfg; + protected volatile CacheConfiguration dfltCacheCfg; /** Enable default data region persistence. */ protected boolean persistence = true; @@ -239,7 +239,7 @@ public static Optional searchDirectoryRecursively(Path path, String dir) t * @return Ignite instance. * @throws Exception If fails. */ - protected IgniteEx startGridWithCache(CacheConfiguration ccfg, int keys) throws Exception { + protected IgniteEx startGridWithCache(CacheConfiguration ccfg, int keys) throws Exception { return startGridsWithCache(1, ccfg, keys); } @@ -250,7 +250,7 @@ protected IgniteEx startGridWithCache(CacheConfiguration ccfg, * @return Ignite instance. * @throws Exception If fails. */ - protected IgniteEx startGridsWithCache(int grids, CacheConfiguration ccfg, int keys) throws Exception { + protected IgniteEx startGridsWithCache(int grids, CacheConfiguration ccfg, int keys) throws Exception { dfltCacheCfg = ccfg; return startGridsWithCache(grids, keys, Integer::new, ccfg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java new file mode 100644 index 00000000000000..4eaa49d84f2fce --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -0,0 +1,679 @@ +/* + * 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.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Objects; +import java.util.UUID; +import java.util.function.Function; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.binary.BinaryObject; +import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.cache.CacheAtomicityMode; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cluster.ClusterState; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.util.distributed.DistributedProcess; +import org.apache.ignite.internal.util.distributed.SingleNodeMessage; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.testframework.GridTestUtils; +import org.junit.Ignore; +import org.junit.Test; + +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.END_SNAPSHOT_RESTORE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT_RESTORE; +import static org.apache.ignite.testframework.GridTestUtils.runAsync; + +/** + * Snapshot restore tests. + */ +public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTest { + /** Timeout. */ + private static final long MAX_AWAIT_MILLIS = 15_000; + + private static final String BIN_TYPE_NAME = "customType"; + + protected CacheConfiguration[] cacheCfgs; + + protected Function valueBuilder = new IndexedValueBuilder(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(name); + + if (cacheCfgs != null) + cfg.setCacheConfiguration(cacheCfgs); + else { + dfltCacheCfg.setSqlIndexMaxInlineSize(255); + dfltCacheCfg.setQueryEntities( + Arrays.asList(queryEntity(BIN_TYPE_NAME), queryEntity(IndexedObject.class.getName()))); + } + + return cfg; + } + + private QueryEntity queryEntity(String typeName) { + return new QueryEntity() + .setKeyType("java.lang.Integer") + .setValueType(typeName) + .setFields(new LinkedHashMap<>(F.asMap("id", Integer.class.getName(), "name", String.class.getName()))) + .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); + } + +// /** @throws Exception If fails. */ +// @Before +// @Override public void beforeTestSnapshot() throws Exception { +// super.beforeTestSnapshot(); +// } + + /** {@inheritDoc} */ + @Override public void afterTestSnapshot() throws Exception { + stopAllGrids(); + } + + /** @throws Exception If fails. */ + @Test + public void testBasicClusterSnapshotRestore() throws Exception { + int keysCnt = 10_000; + + IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); + + ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, keysCnt); + } + + /** @throws Exception If fails. */ + @Test + public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { + int keysCnt = 10_000; + + valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + + IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); + + // remove metadata + int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); + + ignite.context().cacheObjects().removeType(typeId); + + forceCheckpoint(); + + ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, keysCnt); + } + + /** @throws Exception If fails. */ + @Test + public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception { + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + ignite.cluster().state(ClusterState.INACTIVE); + + IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + GridTestUtils.assertThrowsAnyCause( + log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteException.class, "The cluster should be active"); + } + + /** @throws Exception If fails. */ + @Test + public void testRestoreWithMissedPartitions() throws Exception { + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg.setBackups(0)); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + putKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); + + forceCheckpoint(); + + stopGrid(1); + + IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + GridTestUtils.assertThrowsAnyCause( + log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteCheckedException.class, "not all partitions available"); + + startGrid(1); + + IgniteFuture fut1 = ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + GridTestUtils.assertThrowsAnyCause( + log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, "Cache group \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); + + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + } + + /** @throws Exception If fails. */ + @Test + public void testClusterSnapshotRestoreDiffTopology() throws Exception { + int nodesCnt = 4; + + int keysCnt = 10_000; + + valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + + IgniteEx ignite = startGridsWithCache(nodesCnt - 2, keysCnt, valueBuilder, dfltCacheCfg); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + startGrid(nodesCnt - 2); + startGrid(nodesCnt - 1); + + resetBaselineTopology(); + + awaitPartitionMapExchange(); + + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + // remove metadata + int typeId = grid(nodesCnt - 1).context().cacheObjects().typeId(BIN_TYPE_NAME); + + grid(nodesCnt - 1).context().cacheObjects().removeType(typeId); + + forceCheckpoint(); + + ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + + IgniteCache cache = grid(nodesCnt - 1).cache(dfltCacheCfg.getName()).withKeepBinary(); + + assertTrue(cache.indexReadyFuture().isDone()); + + awaitPartitionMapExchange(); + + checkCacheKeys(cache, keysCnt); + } + + /** @throws Exception If fails. */ + @Test + public void testRestoreSharedCacheGroup() throws Exception { + String grpName = "shared"; + String cacheName1 = "cache1"; + String cacheName2 = "cache2"; + + CacheConfiguration cacheCfg1 = txCacheConfig(new CacheConfiguration<>(cacheName1)).setGroupName(grpName); + CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration<>(cacheName2)) + .setAtomicityMode(CacheAtomicityMode.ATOMIC).setGroupName(grpName); + + cacheCfgs = new CacheConfiguration[] {cacheCfg1, cacheCfg2}; + + IgniteEx ignite = startGrids(2); + + ignite.cluster().state(ClusterState.ACTIVE); + + IgniteCache cache1 = ignite.cache(cacheName1); + IgniteCache cache2 = ignite.cache(cacheName2); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) { + cache1.put(i, i); + cache2.put(i, i); + } + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + cache1.destroy(); + + awaitPartitionMapExchange(); + + IgniteSnapshotManager snapshotMgr = ignite.context().cache().context().snapshotMgr(); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(MAX_AWAIT_MILLIS), + IllegalArgumentException.class, + "Cache group(s) \"" + cacheName1 + ", " + cacheName2 + "\" not found in snapshot \"" + SNAPSHOT_NAME + "\"" + ); + + cache2.destroy(); + + awaitPartitionMapExchange(); + + snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(MAX_AWAIT_MILLIS); + + cache1 = ignite.cache(cacheName1); + cache2 = ignite.cache(cacheName2); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) { + assertEquals(i, cache1.get(i)); + assertEquals(i, cache2.get(i)); + } + } + + /** @throws Exception If fails. */ + @Test + public void testIncompatibleMetasUpdate() throws Exception { + valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + + IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); + + ignite.context().cacheObjects().removeType(typeId); + + BinaryObject[] objs = new BinaryObject[CACHE_KEYS_RANGE]; + + IgniteCache cache1 = createCacheWithBinaryType(ignite, "cache1", n -> { + BinaryObjectBuilder builder = ignite.binary().builder(BIN_TYPE_NAME); + + builder.setField("id", n); + + objs[n] = builder.build(); + + return objs[n]; + }); + + IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + fut.get(MAX_AWAIT_MILLIS); + + // Ensure that existing type has been updated + BinaryType type = ignite.context().cacheObjects().metadata(typeId); + + assertTrue(type.fieldNames().contains("name")); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + assertEquals(objs[i], cache1.get(i)); + + cache1.destroy(); + + grid(0).cache(dfltCacheCfg.getName()).destroy(); + + ignite.context().cacheObjects().removeType(typeId); + + // Create cache with incompatible binary type + cache1 = createCacheWithBinaryType(ignite, "cache1", n -> { + BinaryObjectBuilder builder = ignite.binary().builder(BIN_TYPE_NAME); + + builder.setField("id", UUID.randomUUID()); + + objs[n] = builder.build(); + + return objs[n]; + }); + + final IgniteFuture fut0 = ignite.context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> fut0.get(MAX_AWAIT_MILLIS), + IgniteException.class, + "Snapshot restore operation was rejected. Incompatible binary types found" + ); + + ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + assertEquals(objs[i], cache1.get(i)); + } + + private IgniteCache createCacheWithBinaryType(Ignite ignite, String cacheName, Function valBuilder) { + IgniteCache cache = ignite.createCache(new CacheConfiguration<>(cacheName)).withKeepBinary(); + + for (int i = 0; i < CACHE_KEYS_RANGE; i++) + cache.put(i, valBuilder.apply(i)); + + return cache; + } + + @Test + @Ignore + public void testParallelCacheStartWithTheSameName() throws Exception { + int keysCnt = 10_000; + + IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + + IgniteFuture fut = waitForBlockOnRestore(spi, START_SNAPSHOT_RESTORE); + + IgniteCache cache = ignite.createCache(dfltCacheCfg); + + spi.stopBlock(); + + fut.get(MAX_AWAIT_MILLIS); + + checkCacheKeys(grid(0).cache(dfltCacheCfg.getName()), keysCnt); + } + + /** @throws Exception If fails. */ + @Test + public void testRollbackOnNodeFail() throws Exception { + checkBaselineChange(true); + } + + /** @throws Exception If fails. */ + @Test + public void testNodeJoin() throws Exception { + checkBaselineChange(false); + } + + private void checkBaselineChange(boolean stopNode) throws Exception { + int keysCnt = 10_000; + + IgniteEx ignite = startGridsWithSnapshot(4, keysCnt); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); + + IgniteFuture fut = waitForBlockOnRestore(spi, END_SNAPSHOT_RESTORE); + + if (stopNode) { + runAsync(() -> stopGrid(3, true)); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> fut.get(MAX_AWAIT_MILLIS), + IgniteException.class, + "Snapshot restore operation was rejected. Baseline node has left the cluster" + ); + + ensureCacheDirEmpty(3, dfltCacheCfg.getName()); + + return; + } + + startGrid(4); + + resetBaselineTopology(); + + spi.stopBlock(); + + fut.get(MAX_AWAIT_MILLIS); + + IgniteCache cache = grid(4).cache(dfltCacheCfg.getName()); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, keysCnt); + } + + private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCheckedException { + for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { + IgniteEx grid = grid(nodeIdx); + + File dir = resolveCacheDir(grid, cacheName); + + String errMsg = String.format("%s, dir=%s, exists=%b, files=%s", + grid.name(), dir, dir.exists(), Arrays.toString(dir.list())); + + assertTrue(errMsg, !dir.exists() || dir.list().length == 0); + } + } + + private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws IgniteCheckedException { + File workDIr = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); + + String nodeDirName = ignite.context().pdsFolderResolver().resolveFolders().folderName() + File.separator; + + File cacheDir = new File(workDIr, nodeDirName + CACHE_DIR_PREFIX + cacheOrGrpName); + + if (cacheDir.exists()) + return cacheDir; + + return new File(workDIr, nodeDirName + CACHE_GRP_DIR_PREFIX + cacheOrGrpName); + } + + private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { + IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder, dfltCacheCfg); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + return ignite; + } + + private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcess.DistributedProcessType restorePhase) throws InterruptedException { + spi.blockMessages((node, msg) -> + msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == restorePhase.ordinal()); + + IgniteFuture fut = grid(0).context().cache().context().snapshotMgr(). + restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + spi.waitForBlocked(); + + return fut; + } + + /** @throws Exception If fails. */ + @Test + // todo + @Ignore + public void testActivateFromClientWhenRestoring() throws Exception { + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg); + + IgniteEx client = startClientGrid("client"); + + client.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + putKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); + + client.cluster().state(ClusterState.INACTIVE); + + IgniteSnapshotManager snapshotMgr = grid(1).context().cache().context().snapshotMgr(); + + // todo block distribprocess and try to activate cluster + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + + spi.blockMessages((node, msg) -> { + if (msg instanceof SingleNodeMessage) + return true; + + System.out.println(">xxx> " + node.id()); + + return false; + }); + + IgniteFuture fut = snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + spi.waitForBlocked(); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> { + client.cluster().state(ClusterState.ACTIVE); + + return null; + }, + IllegalStateException.class, + "The cluster cannot be activated until the snapshot restore operation is complete." + ); + + spi.stopBlock(); + + fut.get(MAX_AWAIT_MILLIS); + + client.cluster().state(ClusterState.ACTIVE); + + checkCacheKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + } + +// @Test +// public void testPreventRecoveryOnRestoredCacheGroup() throws Exception { +// IgniteEx ignite = startGridsWithCache(2, dfltCacheCfg, CACHE_KEYS_RANGE); +// +// resetBaselineTopology(); +// +// ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); +// +// enableCheckpoints(G.allGrids(), false); +// +// putKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); +// +// stopAllGrids(); +// +// ignite = startGrid(0); +// startGrid(1); +// +// ignite.cluster().state(ClusterState.ACTIVE); +// +// ignite.cache(dfltCacheCfg.getName()).destroy(); +// +// awaitPartitionMapExchange(); +// +// ignite.context().cache().context().snapshotMgr(). +// restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); +// +// stopAllGrids(); +// +// ignite = startGrid(0); +// startGrid(1); +// +// ignite.cluster().state(ClusterState.ACTIVE); +// +// checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); +// } + + private void checkCacheKeys(IgniteCache testCache, int keysCnt) { + assertEquals(keysCnt, testCache.size()); + + for (int i = 0; i < keysCnt; i++) + assertEquals(valueBuilder.apply(i), testCache.get(i)); + } + + private void putKeys(IgniteCache cache, int startIdx, int cnt) { + for (int i = startIdx; i < (startIdx + cnt); i++) + cache.put(i, i); + } + + /** */ + private static class IntValueBuilder implements Function { + /** {@inheritDoc} */ + @Override public Object apply(Integer key) { + return key; + } + } + + private static class IndexedValueBuilder implements Function { + /** {@inheritDoc} */ + @Override public Object apply(Integer key) { + return new IndexedObject(key, "Person number #" + key); + } + } + + /** */ + private static class IndexedObject { + /** Id. */ + @QuerySqlField(index = true) + private final int id; + + /** Name. */ + @QuerySqlField(index = true) + private final String name; + + /** + * @param id Id. + */ + public IndexedObject(int id, String name) { + this.id = id; + this.name = name; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + IndexedObject obj = (IndexedObject)o; + + return id == obj.id && Objects.equals(name, obj.name); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(name, id); + } + } + + /** */ + private class BinaryValueBuilder implements Function { + /** Ignite node index. */ + private final int nodeIdx; + + /** Binary type name. */ + private final String typeName; + + /** + * @param nodeIdx Ignite node index. + * @param typeName Binary type name. + */ + BinaryValueBuilder(int nodeIdx, String typeName) { + this.nodeIdx = nodeIdx; + this.typeName = typeName; + } + + /** {@inheritDoc} */ + @Override public Object apply(Integer key) { + BinaryObjectBuilder builder = grid(nodeIdx).binary().builder(typeName); + + builder.setField("id", key); + builder.setField("name", String.valueOf(key)); + + return builder.build(); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java index 593f5dfab9ea72..72f81a8465f494 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java @@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IndexingMultithreadedLoadContinuousRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.LongDestroyDurableBackgroundTaskTest; import org.apache.ignite.internal.processors.cache.persistence.db.MultipleParallelCacheDeleteDeadlockTest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshoRestoreSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotCheckWithIndexesTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotWithIndexesTest; import org.apache.ignite.internal.processors.database.IgniteDbMultiNodeWithIndexingPutGetTest; @@ -66,7 +67,8 @@ CacheGroupReencryptionTest.class, IgnitePdsIndexingDefragmentationTest.class, StopRebuildIndexTest.class, - ForceRebuildIndexTest.class + ForceRebuildIndexTest.class, + IgniteClusterSnapshoRestoreSelfTest.class }) public class IgnitePdsWithIndexingTestSuite { } From c0a4b265b19c578fde0cf9a888d552e1c6f1ab03 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 14 Jan 2021 21:03:04 +0300 Subject: [PATCH 02/98] IGNITE-13805 Prevent start cache that is currently restoring. --- .../apache/ignite/internal/IgniteKernal.java | 2 + .../processors/cache/ClusterCachesInfo.java | 11 ++ .../cache/DynamicCacheChangeRequest.java | 10 ++ .../processors/cache/GridCacheProcessor.java | 20 ++- .../GridDhtPartitionsExchangeFuture.java | 2 +- .../snapshot/IgniteSnapshotManager.java | 5 +- .../SnapshotRestoreCacheGroupProcess.java | 109 +++++++++++--- .../SnapshotRestorePrepareResponse.java | 14 +- .../IgniteDynamicCacheStartSelfTest.java | 4 +- .../IgniteClusterSnapshoRestoreSelfTest.java | 136 +++++++++--------- 10 files changed, 207 insertions(+), 106 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index f280ce541e24f8..7b43a5f2fd7c50 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -3240,6 +3240,7 @@ public IgniteInternalCache getCache(String name) { ctx.cache().dynamicStartCaches(cacheCfgs, true, true, + false, false).get(); List createdCaches = new ArrayList<>(cacheCfgs.size()); @@ -3336,6 +3337,7 @@ public IgniteInternalCache getCache(String name) { ctx.cache().dynamicStartCaches(cacheCfgs, false, true, + false, false).get(); List createdCaches = new ArrayList<>(cacheCfgs.size()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index ffe51c8796f949..ce5daf28b69017 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -55,6 +55,7 @@ import org.apache.ignite.internal.managers.systemview.walker.CacheViewWalker; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager; import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateFinishMessage; import org.apache.ignite.internal.processors.cluster.ChangeGlobalStateMessage; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; @@ -1007,6 +1008,16 @@ else if (encMgr.masterKeyDigest() != null && } } + if (err == null && !req.restoredCache()) { + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + + String conflictingName; + + if (snapshotMgr.isCacheGroupRestoring(conflictingName = cacheName) || + ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheGroupRestoring(conflictingName))) + err = new IgniteCheckedException("Cache start failed. A cache named \"" + conflictingName + "\" is currently being restored from a snapshot."); + } + if (err != null) { if (persistedCfgs) res.errs.add(err); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java index 88d44cc23090f9..f6817fe36ebff4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java @@ -107,6 +107,8 @@ public class DynamicCacheChangeRequest implements Serializable { /** Cache configuration enrichment. */ private CacheConfigurationEnrichment cacheCfgEnrichment; + private boolean restoredCache; + /** * @param reqId Unique request ID. * @param cacheName Cache stop name. @@ -480,6 +482,14 @@ public void masterKeyDigest(@Nullable byte[] masterKeyDigest) { return masterKeyDigest; } + public void restoredCache(boolean restoredCache) { + this.restoredCache = restoredCache; + } + + public boolean restoredCache() { + return restoredCache; + } + /** * @return Cache configuration enrichment. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 507a35ad5cdb91..c11081e736acc1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3520,7 +3520,8 @@ public IgniteInternalFuture dynamicStartCache( false, null, ccfg != null && ccfg.isEncryptionEnabled() ? grpKeys.iterator().next() : null, - ccfg != null && ccfg.isEncryptionEnabled() ? masterKeyDigest : null); + ccfg != null && ccfg.isEncryptionEnabled() ? masterKeyDigest : null, + false); if (req != null) { if (req.clientStartOnly()) @@ -3714,14 +3715,16 @@ public IgniteInternalFuture dynamicStartCaches( Collection ccfgList, boolean failIfExists, boolean checkThreadTx, - boolean disabledAfterStart + boolean disabledAfterStart, + boolean restoredCache ) { return dynamicStartCachesByStoredConf( ccfgList.stream().map(StoredCacheData::new).collect(toList()), failIfExists, checkThreadTx, disabledAfterStart, - null); + null, + restoredCache); } /** @@ -3739,7 +3742,8 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean failIfExists, boolean checkThreadTx, boolean disabledAfterStart, - IgniteUuid restartId + IgniteUuid restartId, + boolean restoredCache ) { if (checkThreadTx) { sharedCtx.tm().checkEmptyTransactions(() -> { @@ -3774,7 +3778,8 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( disabledAfterStart, ccfg.queryEntities(), ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null, - ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null); + ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null, + restoredCache); if (req != null) { if (req.clientStartOnly()) { @@ -5044,7 +5049,8 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( boolean disabledAfterStart, @Nullable Collection qryEntities, @Nullable byte[] encKey, - @Nullable byte[] masterKeyDigest + @Nullable byte[] masterKeyDigest, + boolean restoredCache ) throws IgniteCheckedException { DynamicCacheDescriptor desc = cacheDescriptor(cacheName); @@ -5062,6 +5068,8 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( req.restartId(restartId); + req.restoredCache(restoredCache); + if (ccfg != null) { cloneCheckSerializable(ccfg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index e63d78aaf93019..ca051356709ea7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -1162,7 +1162,7 @@ private void ensureClientCachesStarted() { } if (!notStartedCacheConfigs.isEmpty()) - cacheProcessor.dynamicStartCaches(notStartedCacheConfigs, false, false, false); + cacheProcessor.dynamicStartCaches(notStartedCacheConfigs, false, false, false, false); } /** 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 f456c8c9c63768..c7fa336e7abca1 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 @@ -760,8 +760,9 @@ public boolean isSnapshotCreating() { } } - public boolean isSnapshotRestoring() { - return restoreCacheGrpProcess.inProgress(); + public boolean isCacheGroupRestoring(String grpName) { + // todo check shared caches names + return restoreCacheGrpProcess.inProgress(grpName); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index e2e427734bfcf4..b246de2ace192b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -24,6 +24,7 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -35,7 +36,6 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; @@ -62,6 +62,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.END_SNAPSHOT_RESTORE; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT_RESTORE; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.UNDO_SNAPSHOT_RESTORE; @@ -153,6 +154,15 @@ public boolean inProgress() { return !fut0.isDone() && fut0.request() != null; } + public boolean inProgress(String cacheName) { + RestoreSnapshotFuture fut0 = fut; + + if (fut0.isDone() || fut0.request() == null) + return false; + + return fut0.request().groups().contains(cacheName) || fut0.sharedCacheNames().contains(cacheName); + } + /** * Node left callback. * @@ -228,7 +238,7 @@ private IgniteInternalFuture prepare(SnapshotRes Set parts = new HashSet<>(); - List> cacheCfgs = new ArrayList<>(1); + List cacheCfgs = new ArrayList<>(1); for (File file : cacheDir.listFiles()) { if (file.isDirectory()) @@ -245,7 +255,9 @@ else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { } } - return new CacheGroupSnapshotDetails(grpName, cacheCfgs, parts); + boolean sharedGrp = cacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX); + + return new CacheGroupSnapshotDetails(grpName, sharedGrp, cacheCfgs, parts); } private void finishPrepare(UUID reqId, Map res, Map errs) { @@ -265,18 +277,21 @@ private void finishPrepare(UUID reqId, Map List notFoundGroups = new ArrayList<>(fut.request().groups()); + Set sharedCacheNames = new HashSet<>(); + try { - Collection grpsDetails = mergeDetails(res); + Collection grpsDetails = mergeNodeResults(res); - List cacheCfgs = new ArrayList<>(); + List cacheCfgs = new ArrayList<>(); for (CacheGroupSnapshotDetails grpDetails : grpsDetails) { - CacheConfiguration ccfg = F.first(grpDetails.configs()); + StoredCacheData cdata = F.first(grpDetails.configs()); - if (ccfg == null) + if (cdata == null) continue; - int reqParts = ccfg.getAffinity().partitions(); + + int reqParts = cdata.config().getAffinity().partitions(); int availParts = grpDetails.parts().size(); if (reqParts != availParts) { @@ -286,14 +301,23 @@ private void finishPrepare(UUID reqId, Map notFoundGroups.remove(grpDetails.groupName()); - cacheCfgs.addAll(grpDetails.configs()); + for (StoredCacheData cacheData : grpDetails.configs()) { + String cacheName = cacheData.config().getName(); - CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(grpDetails.groupName())); + if (grpDetails.shared()) + sharedCacheNames.add(cacheName); - if (desc != null) { - throw new IllegalStateException("Cache group \"" + desc.cacheOrGroupName() + - "\" should be destroyed manually before perform restore operation."); + cacheCfgs.add(cacheData); + + CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheName)); + + if (desc != null) { + throw new IllegalStateException("Cache \"" + desc.cacheOrGroupName() + + "\" should be destroyed manually before perform restore operation."); + } } + + } if (!notFoundGroups.isEmpty()) { @@ -315,6 +339,7 @@ private void finishPrepare(UUID reqId, Map } fut.startConfigs(cacheCfgs); + fut.sharedCacheNames(sharedCacheNames); } catch (Exception e) { fut.onDone(e); @@ -326,7 +351,7 @@ private void finishPrepare(UUID reqId, Map performRestoreProc.start(reqId, fut.request()); } - private Collection mergeDetails(Map responses) { + private Collection mergeNodeResults(Map responses) { Map> globalDetails = new HashMap<>(); for (Map.Entry entry : responses.entrySet()) { @@ -370,6 +395,16 @@ private IgniteInternalFuture perform(SnapshotRes return errResponse("Unknown snapshot restore operation was rejected."); try { + // Double check that cache was not started after first phase. + for (String grpName : req.groups()) { + CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(grpName)); + + if (desc != null) { + throw new IllegalStateException("Cache group \"" + desc.cacheOrGroupName() + + "\" should be destroyed manually before perform restore operation."); + } + } + RestoreOperationContext opCtx = ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups()); @@ -419,7 +454,7 @@ private void finishPerform(UUID reqId, Map return; } - Collection ccfgs0 = fut0.startConfigs(); + Collection ccfgs0 = fut0.startConfigs(); if (fut0 == null || !fut0.id().equals(reqId) || !fut0.initiator() || F.isEmpty(ccfgs0)) { completeFuture(reqId, errs, fut); @@ -427,8 +462,22 @@ private void finishPerform(UUID reqId, Map return; } - ctx.cache().dynamicStartCaches(ccfgs0, true, true, false). - listen(f -> completeFuture(reqId, errs, fut0)); + // todo check whether the cache has been already started + try { + System.out.println(">xxx> start cache(s)"); + + ctx.cache().dynamicStartCachesByStoredConf(ccfgs0, true, true, false, null, true). + listen(f -> { + System.out.println(">xxx> future completed"); + // todo rollback operation + if (f.error() != null) + f.error().printStackTrace(); + + completeFuture(reqId, errs, fut0); + }); + } catch (Exception e) { + e.printStackTrace(); + } } // todo separate rollback request @@ -474,13 +523,13 @@ private boolean completeFuture(UUID reqId, Map err, RestoreSnap return !F.isEmpty(err) ? fut.onDone(F.firstValue(err)) : fut.onDone(); } - private CacheConfiguration unmarshal(IgniteConfiguration cfg, File cacheDataFile) throws IOException, IgniteCheckedException { + private StoredCacheData unmarshal(IgniteConfiguration cfg, File cacheDataFile) throws IOException, IgniteCheckedException { JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(cfg.getIgniteInstanceName()); try (InputStream stream = new BufferedInputStream(new FileInputStream(cacheDataFile))) { StoredCacheData data = marshaller.unmarshal(stream, U.resolveClassLoader(cfg)); - return data.config(); + return data; } } @@ -508,7 +557,9 @@ public Throwable failure() { return err; } - private volatile Collection cacheCfgsToStart; + private volatile Collection cacheCfgsToStart; + + private volatile Set sharedCacheNames; public SnapshotRestoreRequest request() { return reqRef.get(); @@ -518,7 +569,9 @@ public boolean request(SnapshotRestoreRequest req) { return reqRef.compareAndSet(null, req); } - /** @param id Request ID. */ + /** + * @param initiator A flag indicating that the node is the initiator of the request. + */ RestoreSnapshotFuture(boolean initiator) { this.initiator = initiator; } @@ -538,11 +591,21 @@ public void handleError(Throwable err) { this.err = err; } - public void startConfigs(Collection ccfgs) { + public void startConfigs(Collection ccfgs) { cacheCfgsToStart = ccfgs; } - public Collection startConfigs() { + public void sharedCacheNames(Set sharedCacheNames) { + this.sharedCacheNames = sharedCacheNames; + } + + public Set sharedCacheNames() { + Set sharedCacheNames0 = sharedCacheNames; + + return sharedCacheNames0 == null ? Collections.emptySet() : sharedCacheNames0; + } + + public Collection startConfigs() { return cacheCfgsToStart; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java index 5172ba4104d3b8..22386e995c94db 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Set; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.StoredCacheData; /** * Snapshot restore operation single node validation response. @@ -60,14 +61,17 @@ static class CacheGroupSnapshotDetails implements Serializable { private Set parts; /** Group cache configurations. */ - private List> cfgs; + private List cfgs; + + private boolean shared; /** * @param cfgs Group cache configurations. * @param parts Local partition IDs. */ - public CacheGroupSnapshotDetails(String grpName, List> cfgs, Set parts) { + public CacheGroupSnapshotDetails(String grpName, boolean shared, List cfgs, Set parts) { this.grpName = grpName; + this.shared = shared; this.cfgs = cfgs; this.parts = parts; } @@ -75,7 +79,7 @@ public CacheGroupSnapshotDetails(String grpName, List> /** * @return Group cache configurations. */ - public List> configs() { + public List configs() { return cfgs; } @@ -92,5 +96,9 @@ public Set parts() { public String groupName() { return grpName; } + + public boolean shared() { + return shared; + } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java index 7648ec6650bc80..a1bc9e8359dd98 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java @@ -1440,7 +1440,7 @@ public void testCacheRestartIsAllowedOnlyToItsInititator() throws Exception { StoredCacheData storedCacheData = new StoredCacheData(ccfg); try { - kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, IgniteUuid.randomUuid()).get(); + kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, IgniteUuid.randomUuid(), false).get(); fail(); } @@ -1450,7 +1450,7 @@ public void testCacheRestartIsAllowedOnlyToItsInititator() throws Exception { System.out.println("We couldn't start new cache with wrong restart id."); } - kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, restartId).get(); + kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, restartId, false).get(); System.out.println("We successfully restarted cache with initial restartId."); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 4eaa49d84f2fce..01ccde0e25a414 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -31,7 +31,6 @@ import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryType; -import org.apache.ignite.cache.CacheAtomicityMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.annotations.QuerySqlField; @@ -187,7 +186,7 @@ public void testRestoreWithMissedPartitions() throws Exception { restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, "Cache group \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); + log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, "Cache \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); ignite.cache(dfltCacheCfg.getName()).destroy(); @@ -230,7 +229,7 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { forceCheckpoint(); - ignite.context().cache().context().snapshotMgr(). + ignite.snapshot(). restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); IgniteCache cache = grid(nodesCnt - 1).cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -250,8 +249,7 @@ public void testRestoreSharedCacheGroup() throws Exception { String cacheName2 = "cache2"; CacheConfiguration cacheCfg1 = txCacheConfig(new CacheConfiguration<>(cacheName1)).setGroupName(grpName); - CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration<>(cacheName2)) - .setAtomicityMode(CacheAtomicityMode.ATOMIC).setGroupName(grpName); + CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration<>(cacheName2)).setGroupName(grpName); cacheCfgs = new CacheConfiguration[] {cacheCfg1, cacheCfg2}; @@ -259,13 +257,11 @@ public void testRestoreSharedCacheGroup() throws Exception { ignite.cluster().state(ClusterState.ACTIVE); - IgniteCache cache1 = ignite.cache(cacheName1); - IgniteCache cache2 = ignite.cache(cacheName2); + IgniteCache cache1 = ignite.cache(cacheName1); + putKeys(cache1, 0, CACHE_KEYS_RANGE); - for (int i = 0; i < CACHE_KEYS_RANGE; i++) { - cache1.put(i, i); - cache2.put(i, i); - } + IgniteCache cache2 = ignite.cache(cacheName2); + putKeys(cache2, 0, CACHE_KEYS_RANGE); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); @@ -288,13 +284,8 @@ public void testRestoreSharedCacheGroup() throws Exception { snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(MAX_AWAIT_MILLIS); - cache1 = ignite.cache(cacheName1); - cache2 = ignite.cache(cacheName2); - - for (int i = 0; i < CACHE_KEYS_RANGE; i++) { - assertEquals(i, cache1.get(i)); - assertEquals(i, cache2.get(i)); - } + checkCacheKeys(ignite.cache(cacheName1), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); } /** @throws Exception If fails. */ @@ -376,23 +367,66 @@ private IgniteCache createCacheWithBinaryType(Ignite ignite, St } @Test - @Ignore - public void testParallelCacheStartWithTheSameName() throws Exception { - int keysCnt = 10_000; + public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { + checkCacheStartWithTheSameName(true); + } - IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); + @Test + public void testParallelCacheStartWithTheSameNameOnPerform() throws Exception { + checkCacheStartWithTheSameName(false); + } + + private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { + String grpName = "shared"; + String cacheName = "cache1"; + + dfltCacheCfg = txCacheConfig(new CacheConfiguration(cacheName)).setGroupName(grpName); + + IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, START_SNAPSHOT_RESTORE); + IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? START_SNAPSHOT_RESTORE : END_SNAPSHOT_RESTORE, grpName); + + String msgFormat = "Cache start failed. A cache named \"%s\" is currently being restored from a snapshot."; + + GridTestUtils.assertThrowsAnyCause( + log, + () -> ignite.createCache(grpName), + IgniteCheckedException.class, + String.format(msgFormat, grpName) + ); - IgniteCache cache = ignite.createCache(dfltCacheCfg); + if (prepare) + ignite.createCache(cacheName); + else { + GridTestUtils.assertThrowsAnyCause( + log, + () -> ignite.createCache(cacheName), + IgniteCheckedException.class, + String.format(msgFormat, cacheName) + ); + } spi.stopBlock(); - fut.get(MAX_AWAIT_MILLIS); + // We don't know shared cache names during prepare phase - so we just interrupting process. + if (prepare) { + GridTestUtils.assertThrowsAnyCause( + log, + () -> fut.get(MAX_AWAIT_MILLIS), + IgniteException.class, + "Cache \"" + cacheName + "\" should be destroyed manually before perform restore operation." + ); + + ensureCacheDirEmpty(0, grpName); + ensureCacheDirEmpty(1, grpName); + } + else { + fut.get(MAX_AWAIT_MILLIS); - checkCacheKeys(grid(0).cache(dfltCacheCfg.getName()), keysCnt); + checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); + } } /** @throws Exception If fails. */ @@ -410,11 +444,11 @@ public void testNodeJoin() throws Exception { private void checkBaselineChange(boolean stopNode) throws Exception { int keysCnt = 10_000; - IgniteEx ignite = startGridsWithSnapshot(4, keysCnt); + startGridsWithSnapshot(4, keysCnt); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); - IgniteFuture fut = waitForBlockOnRestore(spi, END_SNAPSHOT_RESTORE); + IgniteFuture fut = waitForBlockOnRestore(spi, END_SNAPSHOT_RESTORE, dfltCacheCfg.getName()); if (stopNode) { runAsync(() -> stopGrid(3, true)); @@ -484,12 +518,12 @@ private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Except return ignite; } - private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcess.DistributedProcessType restorePhase) throws InterruptedException { + private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcess.DistributedProcessType restorePhase, String grpName) throws InterruptedException { spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == restorePhase.ordinal()); - IgniteFuture fut = grid(0).context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut = + grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)); spi.waitForBlocked(); @@ -549,42 +583,6 @@ public void testActivateFromClientWhenRestoring() throws Exception { checkCacheKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } -// @Test -// public void testPreventRecoveryOnRestoredCacheGroup() throws Exception { -// IgniteEx ignite = startGridsWithCache(2, dfltCacheCfg, CACHE_KEYS_RANGE); -// -// resetBaselineTopology(); -// -// ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); -// -// enableCheckpoints(G.allGrids(), false); -// -// putKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); -// -// stopAllGrids(); -// -// ignite = startGrid(0); -// startGrid(1); -// -// ignite.cluster().state(ClusterState.ACTIVE); -// -// ignite.cache(dfltCacheCfg.getName()).destroy(); -// -// awaitPartitionMapExchange(); -// -// ignite.context().cache().context().snapshotMgr(). -// restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); -// -// stopAllGrids(); -// -// ignite = startGrid(0); -// startGrid(1); -// -// ignite.cluster().state(ClusterState.ACTIVE); -// -// checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); -// } - private void checkCacheKeys(IgniteCache testCache, int keysCnt) { assertEquals(keysCnt, testCache.size()); @@ -592,9 +590,9 @@ private void checkCacheKeys(IgniteCache testCache, int keysCnt) assertEquals(valueBuilder.apply(i), testCache.get(i)); } - private void putKeys(IgniteCache cache, int startIdx, int cnt) { + private void putKeys(IgniteCache cache, int startIdx, int cnt) { for (int i = startIdx; i < (startIdx + cnt); i++) - cache.put(i, i); + cache.put(i, valueBuilder.apply(i)); } /** */ From 43364e48aae2ba1cfc9dc22a332cde8e58ce5dd1 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Sat, 16 Jan 2021 11:25:48 +0300 Subject: [PATCH 03/98] IGNITE-13805 (wip) Rollback on cache start. --- .../processors/cache/ClusterCachesInfo.java | 3 + .../processors/cache/GridCacheProcessor.java | 13 ++++ .../snapshot/IgniteSnapshotManager.java | 9 ++- .../SnapshotRestoreCacheGroupProcess.java | 72 +++++++++++-------- .../IgniteClusterSnapshoRestoreSelfTest.java | 43 +++++++++-- 5 files changed, 104 insertions(+), 36 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index ce5daf28b69017..b33f6193382593 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1019,6 +1019,9 @@ else if (encMgr.masterKeyDigest() != null && } if (err != null) { + if (req.restoredCache()) + ctx.cache().context().snapshotMgr().rollbackRestoreLocal(); + if (persistedCfgs) res.errs.add(err); else diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index c11081e736acc1..fabf9007508017 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -4273,6 +4273,9 @@ else if (msg0 instanceof WalStateFinishMessage) if (res == null) res = validateRestartingCaches(node); + if (res == null) + res = validateRestoringCaches(node); + return res; } @@ -4299,6 +4302,16 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { return null; } + private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { + if (ctx.cache().context().snapshotMgr().isCacheGroupRestoring(null)) { + String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; + + return new IgniteNodeValidationResult(node.id(), msg); + } + + return null; + } + /** * @return Keep static cache configuration flag. If {@code true}, static cache configuration will override * configuration persisted on disk. 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 c7fa336e7abca1..43aee9af4ee8f1 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 @@ -760,9 +760,12 @@ public boolean isSnapshotCreating() { } } - public boolean isCacheGroupRestoring(String grpName) { - // todo check shared caches names - return restoreCacheGrpProcess.inProgress(grpName); + public boolean isCacheGroupRestoring(@Nullable String cacheName) { + return restoreCacheGrpProcess.inProgress(cacheName); + } + + public void rollbackRestoreLocal() { + restoreCacheGrpProcess.rollbackLocal(); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index b246de2ace192b..2fcf8a0f021209 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -32,6 +32,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -92,6 +93,8 @@ public class SnapshotRestoreCacheGroupProcess { /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); + private ReentrantLock rollbackLock = new ReentrantLock(); + /** * @param ctx Kernal context. */ @@ -148,19 +151,41 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * * @return {@code True} if cache group restore process is currently running. */ - public boolean inProgress() { + public boolean inProgress(@Nullable String cacheName) { RestoreSnapshotFuture fut0 = fut; - return !fut0.isDone() && fut0.request() != null; + if (fut0.isDone() || fut0.request() == null) + return false; + + return cacheName == null || fut0.request().groups().contains(cacheName) || fut0.sharedCacheNames().contains(cacheName); } - public boolean inProgress(String cacheName) { + public boolean rollbackLocal() { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.request() == null) - return false; +// if (fut0.isDone() || fut0.request() == null) { +// System.out.println(">xxx> fut0 done " + fut0.isDone()); +// +// return false; +// } + + rollbackLock.lock(); + + try { + RestoreOperationContext opCtx = fut0.rollbackContext(); + + if (opCtx == null) + return false; + + // We can only have one rollback context for process. + fut0.rollbackContext(null); + + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(fut0.request().groups(), opCtx); + } finally { + rollbackLock.unlock(); + } - return fut0.request().groups().contains(cacheName) || fut0.sharedCacheNames().contains(cacheName); + return true; } /** @@ -186,7 +211,7 @@ private IgniteInternalFuture prepare(SnapshotRes if (ctx.clientNode()) return new GridFinishedFuture<>(); - if (inProgress()) + if (inProgress(null)) return errResponse(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); if (!ctx.state().clusterState().state().active()) @@ -279,6 +304,8 @@ private void finishPrepare(UUID reqId, Map Set sharedCacheNames = new HashSet<>(); + fut.sharedCacheNames(sharedCacheNames); + try { Collection grpsDetails = mergeNodeResults(res); @@ -290,7 +317,6 @@ private void finishPrepare(UUID reqId, Map if (cdata == null) continue; - int reqParts = cdata.config().getAffinity().partitions(); int availParts = grpDetails.parts().size(); @@ -316,8 +342,6 @@ private void finishPrepare(UUID reqId, Map "\" should be destroyed manually before perform restore operation."); } } - - } if (!notFoundGroups.isEmpty()) { @@ -339,7 +363,6 @@ private void finishPrepare(UUID reqId, Map } fut.startConfigs(cacheCfgs); - fut.sharedCacheNames(sharedCacheNames); } catch (Exception e) { fut.onDone(e); @@ -394,17 +417,9 @@ private IgniteInternalFuture perform(SnapshotRes if (!req.equals(fut.request())) return errResponse("Unknown snapshot restore operation was rejected."); - try { - // Double check that cache was not started after first phase. - for (String grpName : req.groups()) { - CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(grpName)); - - if (desc != null) { - throw new IllegalStateException("Cache group \"" + desc.cacheOrGroupName() + - "\" should be destroyed manually before perform restore operation."); - } - } + rollbackLock.lock(); + try { RestoreOperationContext opCtx = ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups()); @@ -418,6 +433,8 @@ private IgniteInternalFuture perform(SnapshotRes fut0.onDone(e); return new GridFinishedFuture<>(e); + } finally { + rollbackLock.unlock(); } } @@ -464,16 +481,15 @@ private void finishPerform(UUID reqId, Map // todo check whether the cache has been already started try { - System.out.println(">xxx> start cache(s)"); - ctx.cache().dynamicStartCachesByStoredConf(ccfgs0, true, true, false, null, true). listen(f -> { - System.out.println(">xxx> future completed"); - // todo rollback operation - if (f.error() != null) - f.error().printStackTrace(); + if (f.error() != null) { + fut0.onDone(f.error()); + + return; + } - completeFuture(reqId, errs, fut0); + fut0.onDone(); }); } catch (Exception e) { e.printStackTrace(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 01ccde0e25a414..556923762b1717 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -39,11 +39,13 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; import org.junit.Ignore; import org.junit.Test; @@ -444,7 +446,7 @@ public void testNodeJoin() throws Exception { private void checkBaselineChange(boolean stopNode) throws Exception { int keysCnt = 10_000; - startGridsWithSnapshot(4, keysCnt); + IgniteEx ignite = startGridsWithSnapshot(4, keysCnt); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); @@ -465,21 +467,52 @@ private void checkBaselineChange(boolean stopNode) throws Exception { return; } - startGrid(4); - - resetBaselineTopology(); + GridTestUtils.assertThrowsAnyCause( + log, + () -> startGrid(4), + IgniteSpiException.class, + "Joining node during caches restore is not allowed" + ); spi.stopBlock(); fut.get(MAX_AWAIT_MILLIS); - IgniteCache cache = grid(4).cache(dfltCacheCfg.getName()); + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); assertTrue(cache.indexReadyFuture().isDone()); checkCacheKeys(cache, keysCnt); } + @Test + public void testClusterStateChangeActiveReadonly() throws Exception { + Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + + IgniteFuture fut = waitForBlockOnRestore(spi, START_SNAPSHOT_RESTORE, dfltCacheCfg.getName()); + + ignite.cluster().state(ClusterState.ACTIVE_READ_ONLY); + + spi.stopBlock(); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> fut.get(MAX_AWAIT_MILLIS), + IgniteClusterReadOnlyException.class, + "Failed to perform start cache operation (cluster is in read-only mode)" + ); + + ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + + ignite.cluster().state(ClusterState.ACTIVE); + + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + } + private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCheckedException { for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { IgniteEx grid = grid(nodeIdx); From 0034c2d1933ac680d87926f7a6d16c558f92e7f9 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 18 Jan 2021 17:38:06 +0300 Subject: [PATCH 04/98] IGNITE-13805 (wip) Cluster state change processing. --- .../cache/GridCacheSharedContext.java | 2 + .../snapshot/IgniteSnapshotManager.java | 15 +++++- .../SnapshotRestoreCacheGroupProcess.java | 21 +++++++++ .../IgniteClusterSnapshoRestoreSelfTest.java | 46 ++++++++++++++----- 4 files changed, 72 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java index f3495ac74516a4..cb513f3b0a67c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java @@ -293,6 +293,8 @@ public GridCacheSharedContext( stateAwareMgrs.add(snpMgr); + stateAwareMgrs.add(snapshotMgr); + for (PluginProvider prv : kernalCtx.plugins().allProviders()) if (prv instanceof IgniteChangeGlobalStateSupport) stateAwareMgrs.add(((IgniteChangeGlobalStateSupport)prv)); 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 43aee9af4ee8f1..73100a09ca4e54 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 @@ -109,6 +109,7 @@ import org.apache.ignite.internal.processors.cache.tree.DataRow; import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; +import org.apache.ignite.internal.processors.cluster.IgniteChangeGlobalStateSupport; import org.apache.ignite.internal.processors.marshaller.MappedName; import org.apache.ignite.internal.processors.metric.MetricRegistry; import org.apache.ignite.internal.processors.task.GridInternal; @@ -195,7 +196,7 @@ * */ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter - implements IgniteSnapshot, PartitionsExchangeAware, MetastorageLifecycleListener { + implements IgniteSnapshot, PartitionsExchangeAware, MetastorageLifecycleListener, IgniteChangeGlobalStateSupport { /** File with delta pages suffix. */ public static final String DELTA_SUFFIX = ".delta"; @@ -436,6 +437,8 @@ public static String partDeltaFileName(int partId) { busyLock.block(); try { + restoreCacheGrpProcess.stop("Node is stopping."); + // Try stop all snapshot processing if not yet. for (SnapshotFutureTask sctx : locSnpTasks.values()) sctx.acceptException(new NodeStoppingException(SNP_NODE_STOPPING_ERR_MSG)); @@ -463,6 +466,16 @@ public static String partDeltaFileName(int partId) { } } + /** {@inheritDoc} */ + @Override public void onActivate(GridKernalContext kctx) { + // No-op. + } + + /** {@inheritDoc} */ + @Override public void onDeActivate(GridKernalContext kctx) { + restoreCacheGrpProcess.stop("The cluster has been deactivated."); + } + /** * @param snpDir Snapshot dir. * @param folderName Local node folder name (see {@link U#maskForFileName} with consistent id). diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 2fcf8a0f021209..712c5a74235feb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -207,6 +207,27 @@ public void onNodeLeft(UUID leftNodeId) { } } + public void stop(String msg) { + if (ctx.clientNode()) + return; + + RestoreSnapshotFuture fut0 = fut; + + if (fut0.isDone()) + return; + + SnapshotRestoreRequest req = fut0.request(); + + if (req == null) + return; + + log.warning("Snapshot restore process has been interrupted [grps=" + req.groups() + ']'); + + rollbackLocal(); + + fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + msg)); + } + private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 556923762b1717..b47932c6459be8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -40,7 +40,7 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException; -import org.apache.ignite.internal.util.distributed.DistributedProcess; +import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.U; @@ -486,23 +486,47 @@ private void checkBaselineChange(boolean stopNode) throws Exception { } @Test - public void testClusterStateChangeActiveReadonly() throws Exception { + public void testClusterStateChangeActiveReadonlyDuringPrepare() throws Exception { + checkReadOnlyDuringRestoring(START_SNAPSHOT_RESTORE); + } + + @Test + public void testClusterStateChangeActiveReadonlyDuringPerform() throws Exception { + checkReadOnlyDuringRestoring(END_SNAPSHOT_RESTORE); + } + + private void checkReadOnlyDuringRestoring(DistributedProcessType procType) throws Exception { + checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, procType, IgniteClusterReadOnlyException.class, + "Failed to perform start cache operation (cluster is in read-only mode)"); + } + + @Test + public void testClusterDeactivateOnPrepare() throws Exception { + checkDeactivationDuringRestoring(START_SNAPSHOT_RESTORE); + } + + @Test + public void testClusterDeactivateOnPerform() throws Exception { + checkDeactivationDuringRestoring(END_SNAPSHOT_RESTORE); + } + + private void checkDeactivationDuringRestoring(DistributedProcessType procType) throws Exception { + checkClusterStateChange(ClusterState.INACTIVE, procType, IgniteCheckedException.class, + "The cluster has been deactivated."); + } + + private void checkClusterStateChange(ClusterState state, DistributedProcessType procType, Class expCls, String expMsg) throws Exception { Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, START_SNAPSHOT_RESTORE, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, procType, dfltCacheCfg.getName()); - ignite.cluster().state(ClusterState.ACTIVE_READ_ONLY); + ignite.cluster().state(state); spi.stopBlock(); - GridTestUtils.assertThrowsAnyCause( - log, - () -> fut.get(MAX_AWAIT_MILLIS), - IgniteClusterReadOnlyException.class, - "Failed to perform start cache operation (cluster is in read-only mode)" - ); + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(MAX_AWAIT_MILLIS), expCls, expMsg); ensureCacheDirEmpty(2, dfltCacheCfg.getName()); @@ -551,7 +575,7 @@ private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Except return ignite; } - private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcess.DistributedProcessType restorePhase, String grpName) throws InterruptedException { + private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcessType restorePhase, String grpName) throws InterruptedException { spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == restorePhase.ordinal()); From 751343f412e36e7f00fef9a3244b0731efe83afb Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 18 Jan 2021 18:55:26 +0300 Subject: [PATCH 05/98] IGNITE-13805 (wip) Check cacheId instead of name. --- .../SnapshotRestoreCacheGroupProcess.java | 45 +++++++++---------- 1 file changed, 21 insertions(+), 24 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 712c5a74235feb..445f2ab1dfa46f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -24,7 +24,6 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -46,6 +45,7 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.RestoreOperationContext; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestorePrepareResponse.CacheGroupSnapshotDetails; +import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -157,17 +157,14 @@ public boolean inProgress(@Nullable String cacheName) { if (fut0.isDone() || fut0.request() == null) return false; - return cacheName == null || fut0.request().groups().contains(cacheName) || fut0.sharedCacheNames().contains(cacheName); + return cacheName == null || fut0.containsCache(cacheName); } public boolean rollbackLocal() { RestoreSnapshotFuture fut0 = fut; -// if (fut0.isDone() || fut0.request() == null) { -// System.out.println(">xxx> fut0 done " + fut0.isDone()); -// +// if (fut0.isDone() || fut0.request() == null) // return false; -// } rollbackLock.lock(); @@ -323,10 +320,6 @@ private void finishPrepare(UUID reqId, Map List notFoundGroups = new ArrayList<>(fut.request().groups()); - Set sharedCacheNames = new HashSet<>(); - - fut.sharedCacheNames(sharedCacheNames); - try { Collection grpsDetails = mergeNodeResults(res); @@ -352,7 +345,7 @@ private void finishPrepare(UUID reqId, Map String cacheName = cacheData.config().getName(); if (grpDetails.shared()) - sharedCacheNames.add(cacheName); + fut.addCacheId(CU.cacheId(cacheName)); cacheCfgs.add(cacheData); @@ -596,14 +589,28 @@ public Throwable failure() { private volatile Collection cacheCfgsToStart; - private volatile Set sharedCacheNames; - public SnapshotRestoreRequest request() { return reqRef.get(); } + public Set cacheIds = new GridConcurrentHashSet<>(); + + public boolean containsCache(String name) { + return cacheIds.contains(CU.cacheId(name)); + } + + public void addCacheId(int cacheId) { + cacheIds.add(cacheId); + } + public boolean request(SnapshotRestoreRequest req) { - return reqRef.compareAndSet(null, req); + if (!reqRef.compareAndSet(null, req)) + return false; + + for (String grpName : req.groups()) + cacheIds.add(CU.cacheId(grpName)); + + return true; } /** @@ -632,16 +639,6 @@ public void startConfigs(Collection ccfgs) { cacheCfgsToStart = ccfgs; } - public void sharedCacheNames(Set sharedCacheNames) { - this.sharedCacheNames = sharedCacheNames; - } - - public Set sharedCacheNames() { - Set sharedCacheNames0 = sharedCacheNames; - - return sharedCacheNames0 == null ? Collections.emptySet() : sharedCacheNames0; - } - public Collection startConfigs() { return cacheCfgsToStart; } From 3ebac03efaf8934fb47ba56fea00da7aaf3a3c42 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 18 Jan 2021 21:58:43 +0300 Subject: [PATCH 06/98] IGNITE-13805 Complete future after cache started. --- .../processors/cache/ClusterCachesInfo.java | 7 +- .../processors/cache/GridCacheProcessor.java | 11 +- .../snapshot/IgniteSnapshotManager.java | 52 +++---- .../SnapshotRestoreCacheGroupProcess.java | 127 +++++++++++------- .../SnapshotRestorePrepareResponse.java | 1 - 5 files changed, 117 insertions(+), 81 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index b33f6193382593..e20c29e2004b46 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1013,15 +1013,12 @@ else if (encMgr.masterKeyDigest() != null && String conflictingName; - if (snapshotMgr.isCacheGroupRestoring(conflictingName = cacheName) || - ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheGroupRestoring(conflictingName))) + if (snapshotMgr.isCacheRestoring(conflictingName = cacheName) || + ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheRestoring(conflictingName))) err = new IgniteCheckedException("Cache start failed. A cache named \"" + conflictingName + "\" is currently being restored from a snapshot."); } if (err != null) { - if (req.restoredCache()) - ctx.cache().context().snapshotMgr().rollbackRestoreLocal(); - if (persistedCfgs) res.errs.add(err); else diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index fabf9007508017..a8c2e16be1726d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -2987,6 +2987,11 @@ public void completeCacheStartFuture(DynamicCacheChangeRequest req, boolean succ if (fut != null) fut.onDone(success, err); } + + if (req.restoredCache()) { + ctx.cache().context().snapshotMgr().afterRestoredCacheStarted(req.cacheName(), + req.startCacheConfiguration().getGroupName(), err); + } } /** @@ -4302,8 +4307,12 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { return null; } + /** + * @param node Joining node to validate. + * @return Node validation result if there was an issue with the joining node, {@code null} otherwise. + */ private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { - if (ctx.cache().context().snapshotMgr().isCacheGroupRestoring(null)) { + if (ctx.cache().context().snapshotMgr().isCacheRestoring(null)) { String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; return new IgniteNodeValidationResult(node.id(), msg); 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 73100a09ca4e54..f5682efbc1b72a 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 @@ -53,7 +53,6 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; import java.util.function.Function; @@ -773,12 +772,12 @@ public boolean isSnapshotCreating() { } } - public boolean isCacheGroupRestoring(@Nullable String cacheName) { + public boolean isCacheRestoring(@Nullable String cacheName) { return restoreCacheGrpProcess.inProgress(cacheName); } - public void rollbackRestoreLocal() { - restoreCacheGrpProcess.rollbackLocal(); + public void afterRestoredCacheStarted(String cacheName, @Nullable String grpName, @Nullable Throwable err) { + restoreCacheGrpProcess.handleCacheStart(cacheName, grpName, err); } /** @@ -1100,7 +1099,7 @@ public List readSnapshotMetadatas(String snpName) { return restoreCacheGrpProcess.start(snpName, grpNames); } - protected void ensureMetaCanBeMerged(String snpName) throws IgniteCheckedException, IOException { + protected void checkMetaCompatibility(String snpName) throws IgniteCheckedException, IOException { String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); @@ -1130,9 +1129,11 @@ protected void ensureMetaCanBeMerged(String snpName) throws IgniteCheckedExcepti } } - protected RestoreOperationContext restoreCacheGroupsLocal(String snpName, Collection grpNames) throws IgniteCheckedException { - RestoreOperationContext opCtx = new RestoreOperationContext(); - + protected IgniteInternalFuture restoreCacheGroupsLocal( + String snpName, + Collection grpNames, + RestoreOperationContext opCtx + ) throws IgniteCheckedException { String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); @@ -1143,7 +1144,7 @@ protected RestoreOperationContext restoreCacheGroupsLocal(String snpName, Collec File snapshotMetadataDir = new File(workDIr, subPath); if (!snapshotMetadataDir.exists()) - return opCtx; + return new GridFinishedFuture<>(); // restore metadata CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); @@ -1163,15 +1164,24 @@ protected RestoreOperationContext restoreCacheGroupsLocal(String snpName, Collec } } + GridFutureAdapter updateMetaFut = new GridFutureAdapter<>(); + // todo should register only from one node and validate result if (!F.isEmpty(metas)) { - Future updateMetaFut = cctx.kernalContext().getSystemExecutorService().submit(() -> { - for (BinaryMetadata meta : metas) - procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); - }); + cctx.kernalContext().getSystemExecutorService().submit(() -> { + try { + for (BinaryMetadata meta : metas) + procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); - opCtx.updateMetaFuture(updateMetaFut); + updateMetaFut.onDone(); + } + catch (Exception e) { + updateMetaFut.onDone(e); + } + }); } + else + updateMetaFut.onDone(); for (String grpName : grpNames) { File cacheDir = resolveCacheDir(grpName); @@ -1208,7 +1218,7 @@ protected RestoreOperationContext restoreCacheGroupsLocal(String snpName, Collec } } - return opCtx; + return updateMetaFut; } protected void rollbackRestoreOperation(Collection grps, RestoreOperationContext opCtx) { @@ -1248,7 +1258,7 @@ private File resolveCacheDir(String cacheOrGrpName) throws IgniteCheckedExceptio protected File resolveSnapshotCacheDir(String snpName, IgniteConfiguration cfg, String cacheName) throws IgniteCheckedException { File workDIr = resolveSnapshotWorkDirectory(cfg); - String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName() + File.separator; + String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + nodeDirName + File.separator; @@ -2189,16 +2199,6 @@ static class RestoreOperationContext { private final List metadataTypes = new ArrayList<>(); - private volatile Future updateMetaFuture; - - public void updateMetaFuture(Future updateMetaFuture) { - this.updateMetaFuture = updateMetaFuture; - } - - public @Nullable Future updateMetaFuture() { - return this.updateMetaFuture; - } - public void cacheGroupFile(String grpName, File newFile) { files.computeIfAbsent(grpName, v -> new ArrayList<>()).add(newFile); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 445f2ab1dfa46f..ae003a36974063 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -90,11 +90,12 @@ public class SnapshotRestoreCacheGroupProcess { /** Logger. */ private final IgniteLogger log; + /** Restore operation lock. */ + private final ReentrantLock rollbackLock = new ReentrantLock(); + /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); - private ReentrantLock rollbackLock = new ReentrantLock(); - /** * @param ctx Kernal context. */ @@ -160,24 +161,57 @@ public boolean inProgress(@Nullable String cacheName) { return cacheName == null || fut0.containsCache(cacheName); } - public boolean rollbackLocal() { + /** + * @param cacheName Started cache name. + * @param grpName Started cache group name. + * @param err Error if any. + */ + public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullable Throwable err) { RestoreSnapshotFuture fut0 = fut; -// if (fut0.isDone() || fut0.request() == null) -// return false; + if (fut0.isDone() || fut0.request() == null) + return; + + String grpName0 = grpName != null ? grpName : cacheName; + + if (err != null && fut.rollbackContext() != null) { + ctx.getSystemExecutorService().submit(() -> { + rollbackChanges(fut0); + + fut0.onDone(err); + }); + + return; + } + if (grpName0 != null) + fut0.pendingStartCaches.remove(grpName0); + + if (fut0.pendingStartCaches.isEmpty()) + fut0.onDone(); + } + + /** + * Rollback changes made by process. + * + * @param fut Restore future/ + * @return {@code True} if changes were rolled back, {@code False} if changes have been already rolled back. + */ + public boolean rollbackChanges(RestoreSnapshotFuture fut) { rollbackLock.lock(); try { - RestoreOperationContext opCtx = fut0.rollbackContext(); + if (fut.isDone()) + return false; + + RestoreOperationContext opCtx = fut.rollbackContext(); if (opCtx == null) return false; - // We can only have one rollback context for process. - fut0.rollbackContext(null); + fut.rollbackContext(null); - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(fut0.request().groups(), opCtx); + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(fut.request().groups(), opCtx); } finally { rollbackLock.unlock(); } @@ -204,7 +238,12 @@ public void onNodeLeft(UUID leftNodeId) { } } - public void stop(String msg) { + /** + * Abort the currently running restore procedure (if any). + * + * @param reason Interruption reason. + */ + public void stop(String reason) { if (ctx.clientNode()) return; @@ -220,11 +259,15 @@ public void stop(String msg) { log.warning("Snapshot restore process has been interrupted [grps=" + req.groups() + ']'); - rollbackLocal(); + rollbackChanges(fut0); - fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + msg)); + fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + reason)); } + /** + * @param req Snapshot restore request. + * @return Result future. + */ private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -259,7 +302,7 @@ private IgniteInternalFuture prepare(SnapshotRes return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); try { - ctx.cache().context().snapshotMgr().ensureMetaCanBeMerged(req.snapshotName()); + ctx.cache().context().snapshotMgr().checkMetaCompatibility(req.snapshotName()); } catch (BinaryObjectException e) { log.warning(OP_REJECT_MSG + "Incompatible binary types found", e); @@ -273,6 +316,13 @@ private IgniteInternalFuture prepare(SnapshotRes return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); } + /** + * @param snapshotName Snapshot name. + * @param grpName Cache group name. + * @return Details about the locally stored cache group, or {@code null} if cache group (or snapshot) was not found. + * @throws IgniteCheckedException if failed. + * @throws IOException if I/O errors occur during reading cache configurations. + */ private @Nullable CacheGroupSnapshotDetails readCacheGroupDetails(String snapshotName, String grpName) throws IgniteCheckedException, IOException { File cacheDir = ctx.cache().context().snapshotMgr().resolveSnapshotCacheDir(snapshotName, ctx.config(), grpName); @@ -431,15 +481,18 @@ private IgniteInternalFuture perform(SnapshotRes if (!req.equals(fut.request())) return errResponse("Unknown snapshot restore operation was rejected."); + IgniteInternalFuture retFut; + rollbackLock.lock(); try { - RestoreOperationContext opCtx = - ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups()); + RestoreOperationContext opCtx = new RestoreOperationContext(); + + retFut = ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups(), opCtx); fut.rollbackContext(opCtx); - return new GridFinishedFuture<>(new SnapshotRestorePerformResponse()); + return retFut; } catch (IgniteCheckedException e) { RestoreSnapshotFuture fut0 = fut; @@ -455,15 +508,11 @@ private IgniteInternalFuture perform(SnapshotRes private void finishPerform(UUID reqId, Map map, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (!F.isEmpty(errs)) { - completeFuture(reqId, errs, fut0); - - return; - } + Throwable failure = F.first(errs.values()); - Throwable failure = fut0.failure(); + if (failure == null) + failure = fut0.failure(); - // todo not only left nodes if (failure == null && !map.keySet().containsAll(fut0.request().requiredNodes())) { Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); @@ -477,7 +526,7 @@ private void finishPerform(UUID reqId, Map if (failure != null) { if (U.isLocalNodeCoordinator(ctx.discovery())) { - log.info("Starting rollback routine."); + log.info("Starting snapshot restore rollback routine."); rollbackRestoreProc.start(reqId, fut0.request()); } @@ -485,29 +534,8 @@ private void finishPerform(UUID reqId, Map return; } - Collection ccfgs0 = fut0.startConfigs(); - - if (fut0 == null || !fut0.id().equals(reqId) || !fut0.initiator() || F.isEmpty(ccfgs0)) { - completeFuture(reqId, errs, fut); - - return; - } - - // todo check whether the cache has been already started - try { - ctx.cache().dynamicStartCachesByStoredConf(ccfgs0, true, true, false, null, true). - listen(f -> { - if (f.error() != null) { - fut0.onDone(f.error()); - - return; - } - - fut0.onDone(); - }); - } catch (Exception e) { - e.printStackTrace(); - } + if (U.isLocalNodeCoordinator(ctx.discovery())) + ctx.cache().dynamicStartCachesByStoredConf(fut0.startConfigs(), true, true, false, null, true); } // todo separate rollback request @@ -537,7 +565,6 @@ private void finishRollback(UUID reqId, Map cacheCfgsToStart; + private final Set pendingStartCaches = new GridConcurrentHashSet<>(); + public SnapshotRestoreRequest request() { return reqRef.get(); } @@ -607,6 +636,8 @@ public boolean request(SnapshotRestoreRequest req) { if (!reqRef.compareAndSet(null, req)) return false; + pendingStartCaches.addAll(req.groups()); + for (String grpName : req.groups()) cacheIds.add(CU.cacheId(grpName)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java index 22386e995c94db..828203ef509a8b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -20,7 +20,6 @@ import java.io.Serializable; import java.util.List; import java.util.Set; -import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.processors.cache.StoredCacheData; /** From 63a4edcbfaf1ece4c4e9dcd8d4c02db459d1c2a2 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 20 Jan 2021 18:55:06 +0300 Subject: [PATCH 07/98] IGNITE-13805 Rollback only failed to start groups. --- .../snapshot/IgniteSnapshotManager.java | 240 ++++++++++++------ .../SnapshotRestoreCacheGroupProcess.java | 148 ++++++++--- .../SnapshotRestorePerformRequest.java | 22 ++ 3 files changed, 291 insertions(+), 119 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.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 f5682efbc1b72a..aa718d94663ce6 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 @@ -467,7 +467,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onActivate(GridKernalContext kctx) { - // No-op. + restoreCacheGrpProcess.start(); } /** {@inheritDoc} */ @@ -1129,11 +1129,7 @@ protected void checkMetaCompatibility(String snpName) throws IgniteCheckedExcept } } - protected IgniteInternalFuture restoreCacheGroupsLocal( - String snpName, - Collection grpNames, - RestoreOperationContext opCtx - ) throws IgniteCheckedException { + protected void registerSnapshotMetadata(String snpName) throws IgniteCheckedException { String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); @@ -1143,8 +1139,9 @@ protected IgniteInternalFuture restoreCacheGroup File snapshotMetadataDir = new File(workDIr, subPath); + // todo remove if executed from single node if (!snapshotMetadataDir.exists()) - return new GridFinishedFuture<>(); + return; // restore metadata CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); @@ -1164,82 +1161,157 @@ protected IgniteInternalFuture restoreCacheGroup } } - GridFutureAdapter updateMetaFut = new GridFutureAdapter<>(); - // todo should register only from one node and validate result - if (!F.isEmpty(metas)) { - cctx.kernalContext().getSystemExecutorService().submit(() -> { - try { - for (BinaryMetadata meta : metas) - procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); - - updateMetaFut.onDone(); - } - catch (Exception e) { - updateMetaFut.onDone(e); - } - }); - } - else - updateMetaFut.onDone(); + for (BinaryMetadata meta : metas) + procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); + } - for (String grpName : grpNames) { - File cacheDir = resolveCacheDir(grpName); + protected void restoreCacheGroupFiles(String snpName, String grpName, List newFiles) throws IgniteCheckedException { + File cacheDir = resolveCacheDir(grpName); - assert F.isEmpty(cacheDir.list()) : cacheDir; + assert F.isEmpty(cacheDir.list()) : cacheDir; - File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); + File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); - if (!snapshotCacheDir.exists()) { - log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); + if (!snapshotCacheDir.exists()) { + log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); - continue; - } + return; + } - if (!cacheDir.exists()) { - cacheDir.mkdir(); + if (!cacheDir.exists()) { + cacheDir.mkdir(); - opCtx.cacheGroupFile(grpName, cacheDir); - } + newFiles.add(cacheDir); + } + else + if (cacheDir.list().length != 0) + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty [group=" + grpName + ", dir=" + cacheDir + ']'); - try { - for (File snpFile : snapshotCacheDir.listFiles()) { - File target = new File(cacheDir, snpFile.getName()); + try { + for (File snpFile : snapshotCacheDir.listFiles()) { + File target = new File(cacheDir, snpFile.getName()); - log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); + log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); - Files.copy(snpFile.toPath(), target.toPath()); + newFiles.add(target); - opCtx.cacheGroupFile(grpName, target); - } - } - catch (IOException e) { - throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); + Files.copy(snpFile.toPath(), target.toPath()); } } - - return updateMetaFut; + catch (IOException e) { + throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); + } } - protected void rollbackRestoreOperation(Collection grps, RestoreOperationContext opCtx) { - for (String grpName : grps) { - List files = opCtx.cacheGroupFiles(grpName); - - List dirs = new ArrayList<>(); - for (File file : files) { - if (!file.exists()) - continue; - - if (file.isDirectory()) - dirs.add(file); +// protected IgniteInternalFuture restoreCacheGroupsLocal( +// String snpName, +// Collection grpNames, +// RestoreGroupContext opCtx +// ) throws IgniteCheckedException { +// String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); +// +// File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); +// +// String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + +// nodeFolderName; +// +// File snapshotMetadataDir = new File(workDIr, subPath); +// +// if (!snapshotMetadataDir.exists()) +// return new GridFinishedFuture<>(); +// +// // restore metadata +// CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); +// +// List metas = new ArrayList<>(); +// +// Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); +// ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); +// +// for (File file : snapshotMetadataDir.listFiles()) { +// try (FileInputStream in = new FileInputStream(file)) { +// metas.add(U.unmarshal(marshaller, in, clsLdr)); +// } +// catch (Exception e) { +// throw new IgniteCheckedException("Failed to add metadata from file: " + file.getName() + +// "; exception was thrown: " + e.getMessage()); +// } +// } +// +// GridFutureAdapter updateMetaFut = new GridFutureAdapter<>(); +// +// // todo should register only from one node and validate result +// if (!F.isEmpty(metas)) { +// cctx.kernalContext().getSystemExecutorService().submit(() -> { +// try { +// for (BinaryMetadata meta : metas) +// procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); +// +// updateMetaFut.onDone(); +// } +// catch (Exception e) { +// updateMetaFut.onDone(e); +// } +// }); +// } +// else +// updateMetaFut.onDone(); +// +// for (String grpName : grpNames) { +// File cacheDir = resolveCacheDir(grpName); +// +// assert F.isEmpty(cacheDir.list()) : cacheDir; +// +// File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); +// +// if (!snapshotCacheDir.exists()) { +// log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); +// +// continue; +// } +// +// if (!cacheDir.exists()) { +// cacheDir.mkdir(); +// +// opCtx.cacheGroupFile(grpName, cacheDir); +// } +// +// try { +// for (File snpFile : snapshotCacheDir.listFiles()) { +// File target = new File(cacheDir, snpFile.getName()); +// +// log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); +// +// Files.copy(snpFile.toPath(), target.toPath()); +// +// opCtx.cacheGroupFile(grpName, target); +// } +// } +// catch (IOException e) { +// throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); +// } +// } +// +// return updateMetaFut; +// } + + protected void rollbackRestoreOperation(Collection files) { + List dirs = new ArrayList<>(); + + for (File file : files) { + if (!file.exists()) + continue; - file.delete(); - } + if (file.isDirectory()) + dirs.add(file); - for (File dir : dirs) - dir.delete(); + file.delete(); } + + for (File dir : dirs) + dir.delete(); } private File resolveCacheDir(String cacheOrGrpName) throws IgniteCheckedException { @@ -2194,21 +2266,29 @@ public IgniteSnapshotFutureImpl(IgniteInternalFuture fut) { } } - static class RestoreOperationContext { - private final Map> files = new HashMap<>(); - - private final List metadataTypes = new ArrayList<>(); - - public void cacheGroupFile(String grpName, File newFile) { - files.computeIfAbsent(grpName, v -> new ArrayList<>()).add(newFile); - } - - public void metadataType(int typeId) { - metadataTypes.add(typeId); - } - - public List cacheGroupFiles(String grpName) { - return files.get(grpName); - } - } +// static class RestoreGroupContext { +// private final List files = new ArrayList<>(); +// +// public void addFile(File file) { +// files.add(file); +// } +// +// public List files() { +// return files; +// } +// +//// private final List metadataTypes = new ArrayList<>(); +//// +//// public void cacheGroupFile(String grpName, File newFile) { +//// files.computeIfAbsent(grpName, v -> new ArrayList<>()).add(newFile); +//// } +//// +//// public void metadataType(int typeId) { +//// metadataTypes.add(typeId); +//// } +//// +//// public List cacheGroupFiles(String grpName) { +//// return files.get(grpName); +//// } +// } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index ae003a36974063..bb417cfcfa0a0f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; @@ -43,7 +44,6 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.RestoreOperationContext; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestorePrepareResponse.CacheGroupSnapshotDetails; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.distributed.DistributedProcess; @@ -96,6 +96,8 @@ public class SnapshotRestoreCacheGroupProcess { /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); + private volatile boolean stopped; + /** * @param ctx Kernal context. */ @@ -174,9 +176,18 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab String grpName0 = grpName != null ? grpName : cacheName; - if (err != null && fut.rollbackContext() != null) { + PendingStartCacheGroup pendingGrp = fut0.pendingStartCaches.get(grpName0); + + // If any of shared caches has been started - we cannot rollback changes. + if (pendingGrp.caches.remove(cacheName) && err == null) + pendingGrp.canRollback = false; + + if (!pendingGrp.caches.isEmpty()) + return; + + if (pendingGrp.canRollback && err != null && fut.rollbackContext() != null) { ctx.getSystemExecutorService().submit(() -> { - rollbackChanges(fut0); + rollbackChanges(fut0, grpName0); fut0.onDone(err); }); @@ -184,8 +195,7 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab return; } - if (grpName0 != null) - fut0.pendingStartCaches.remove(grpName0); + fut0.pendingStartCaches.remove(grpName0); if (fut0.pendingStartCaches.isEmpty()) fut0.onDone(); @@ -194,24 +204,23 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab /** * Rollback changes made by process. * - * @param fut Restore future/ + * @param fut Restore future. + * @param grpName Cache group name. * @return {@code True} if changes were rolled back, {@code False} if changes have been already rolled back. */ - public boolean rollbackChanges(RestoreSnapshotFuture fut) { + public boolean rollbackChanges(RestoreSnapshotFuture fut, String grpName) { rollbackLock.lock(); try { if (fut.isDone()) return false; - RestoreOperationContext opCtx = fut.rollbackContext(); + List createdFiles = fut.rollbackContext().remove(grpName); - if (opCtx == null) + if (F.isEmpty(createdFiles)) return false; - fut.rollbackContext(null); - - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(fut.request().groups(), opCtx); + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(createdFiles); } finally { rollbackLock.unlock(); } @@ -244,6 +253,8 @@ public void onNodeLeft(UUID leftNodeId) { * @param reason Interruption reason. */ public void stop(String reason) { + stopped = true; + if (ctx.clientNode()) return; @@ -259,11 +270,17 @@ public void stop(String reason) { log.warning("Snapshot restore process has been interrupted [grps=" + req.groups() + ']'); - rollbackChanges(fut0); + for (String grpName : fut0.request().groups()) + rollbackChanges(fut0, grpName); fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + reason)); } + /** */ + public void start() { + stopped = false; + } + /** * @param req Snapshot restore request. * @return Result future. @@ -391,11 +408,16 @@ private void finishPrepare(UUID reqId, Map notFoundGroups.remove(grpDetails.groupName()); + PendingStartCacheGroup pendingGrp = fut.pendingStartCaches.get(grpDetails.groupName()); + for (StoredCacheData cacheData : grpDetails.configs()) { String cacheName = cacheData.config().getName(); - if (grpDetails.shared()) + // todo replace shared with groupName + if (grpDetails.shared()) { fut.addCacheId(CU.cacheId(cacheName)); + pendingGrp.caches.add(cacheName); + } cacheCfgs.add(cacheData); @@ -481,27 +503,48 @@ private IgniteInternalFuture perform(SnapshotRes if (!req.equals(fut.request())) return errResponse("Unknown snapshot restore operation was rejected."); - IgniteInternalFuture retFut; + GridFutureAdapter retFut = new GridFutureAdapter<>(); - rollbackLock.lock(); + // todo check that snapshot exists + ctx.getSystemExecutorService().submit(() -> { + try { + performRestore(req, fut.rollbackContext()); - try { - RestoreOperationContext opCtx = new RestoreOperationContext(); + retFut.onDone(); + } catch (Throwable t) { + retFut.onDone(t); + } + }); - retFut = ctx.cache().context().snapshotMgr().restoreCacheGroupsLocal(req.snapshotName(), req.groups(), opCtx); + return retFut; + } - fut.rollbackContext(opCtx); + private void performRestore(SnapshotRestoreRequest req, RestoreOperationContext opCtx) throws IgniteCheckedException { + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - return retFut; - } catch (IgniteCheckedException e) { - RestoreSnapshotFuture fut0 = fut; +// if (req.ctx.localNodeId() && !stopped) + snapshotMgr.registerSnapshotMetadata(req.snapshotName()); - if (fut0 != null && fut0.id().equals(req.requestId())) - fut0.onDone(e); + for (String grpName : req.groups()) { + rollbackLock.lock(); - return new GridFinishedFuture<>(e); - } finally { - rollbackLock.unlock(); + try { + if (stopped) + return; + + List newFiles = new ArrayList<>(); + + opCtx.put(grpName, newFiles); + + snapshotMgr.restoreCacheGroupFiles(req.snapshotName(), grpName, newFiles); + } catch (IgniteCheckedException e) { + RestoreSnapshotFuture fut0 = fut; + + if (fut0 != null && fut0.id().equals(req.requestId())) + fut0.onDone(e); + } finally { + rollbackLock.unlock(); + } } } @@ -546,11 +589,11 @@ private IgniteInternalFuture rollback(SnapshotR RestoreSnapshotFuture fut0 = fut; // - if (!req.equals(fut.request())) + if (!req.equals(fut0.request())) return errResponse("Unknown snapshot restore operation was rejected [fut=" + fut + ", req=" + req + ']'); - if (fut0.rollbackContext() != null) - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(req.groups(), fut0.rollbackContext()); + for (String grpName : req.groups()) + rollbackChanges(fut0, grpName); return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse()); } @@ -599,6 +642,32 @@ private IgniteInternalFuture errResponse(Exception ex) { return new GridFinishedFuture<>(ex); } + static class PendingStartCacheGroup { + volatile boolean canRollback = true; + + Set caches = new GridConcurrentHashSet<>(); + } + + static class RestoreOperationContext { + private final Map> newGrpFiles = new HashMap<>(); + + public List get(String grpName) { + return newGrpFiles.get(grpName); + } + + public List remove(String grpName) { + return newGrpFiles.remove(grpName); + } + + public boolean isEmpty() { + return newGrpFiles.isEmpty(); + } + + public void put(String grpName, List files) { + newGrpFiles.put(grpName, files); + } + } + /** */ protected static class RestoreSnapshotFuture extends GridFutureAdapter { /** Request ID. */ @@ -606,7 +675,8 @@ protected static class RestoreSnapshotFuture extends GridFutureAdapter { private final AtomicReference reqRef = new AtomicReference<>(); - private volatile RestoreOperationContext rollbackCtx; +// private volatile RestoreGroupContext rollbackCtx; + private final RestoreOperationContext rollbackCtx = new RestoreOperationContext(); private volatile Throwable err; @@ -616,7 +686,8 @@ public Throwable failure() { private volatile Collection cacheCfgsToStart; - private final Set pendingStartCaches = new GridConcurrentHashSet<>(); +// private final Set pendingStartCaches = new GridConcurrentHashSet<>(); + Map pendingStartCaches = new ConcurrentHashMap<>(); public SnapshotRestoreRequest request() { return reqRef.get(); @@ -636,11 +707,14 @@ public boolean request(SnapshotRestoreRequest req) { if (!reqRef.compareAndSet(null, req)) return false; - pendingStartCaches.addAll(req.groups()); +// pendingStartCaches.addAll(req.groups()); - for (String grpName : req.groups()) + for (String grpName : req.groups()) { cacheIds.add(CU.cacheId(grpName)); + pendingStartCaches.put(grpName, new PendingStartCacheGroup()); + } + return true; } @@ -674,10 +748,6 @@ public Collection startConfigs() { return cacheCfgsToStart; } - public void rollbackContext(RestoreOperationContext opCtx) { - rollbackCtx = opCtx; - } - public RestoreOperationContext rollbackContext() { return rollbackCtx; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java new file mode 100644 index 00000000000000..d6163e0b274b9f --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java @@ -0,0 +1,22 @@ +package org.apache.ignite.internal.processors.cache.persistence.snapshot; + +import java.util.Collection; +import java.util.Set; +import java.util.UUID; + +public class SnapshotRestorePerformRequest extends SnapshotRestoreRequest { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + private final UUID updateMetaNodeId; + + public SnapshotRestorePerformRequest(String snpName, Collection grps, Set reqNodes, UUID updateMetaNodeId) { + super(snpName, grps, reqNodes); + + this.updateMetaNodeId = updateMetaNodeId; + } + + public UUID getUpdateMetaNodeId() { + return updateMetaNodeId; + } +} From c1981ab0cc8e98b143f63a518b74535600916b2d Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 20 Jan 2021 20:30:19 +0300 Subject: [PATCH 08/98] IGNITE-13805 Update metadata from single node. --- .../snapshot/IgniteSnapshotManager.java | 93 ----------------- .../SnapshotRestoreCacheGroupProcess.java | 99 ++++++++++++------- .../SnapshotRestorePerformRequest.java | 25 ++++- .../SnapshotRestorePerformResponse.java | 2 +- ...ava => SnapshotRestorePrepareRequest.java} | 11 ++- .../SnapshotRestorePrepareResponse.java | 2 +- .../SnapshotRestoreRollbackRequest.java | 43 ++++++++ .../SnapshotRestoreRollbackResponse.java | 2 +- 8 files changed, 135 insertions(+), 142 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestoreRequest.java => SnapshotRestorePrepareRequest.java} (85%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.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 aa718d94663ce6..d7987a149b4172 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 @@ -1204,99 +1204,6 @@ protected void restoreCacheGroupFiles(String snpName, String grpName, List } } - -// protected IgniteInternalFuture restoreCacheGroupsLocal( -// String snpName, -// Collection grpNames, -// RestoreGroupContext opCtx -// ) throws IgniteCheckedException { -// String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); -// -// File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); -// -// String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + -// nodeFolderName; -// -// File snapshotMetadataDir = new File(workDIr, subPath); -// -// if (!snapshotMetadataDir.exists()) -// return new GridFinishedFuture<>(); -// -// // restore metadata -// CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); -// -// List metas = new ArrayList<>(); -// -// Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); -// ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); -// -// for (File file : snapshotMetadataDir.listFiles()) { -// try (FileInputStream in = new FileInputStream(file)) { -// metas.add(U.unmarshal(marshaller, in, clsLdr)); -// } -// catch (Exception e) { -// throw new IgniteCheckedException("Failed to add metadata from file: " + file.getName() + -// "; exception was thrown: " + e.getMessage()); -// } -// } -// -// GridFutureAdapter updateMetaFut = new GridFutureAdapter<>(); -// -// // todo should register only from one node and validate result -// if (!F.isEmpty(metas)) { -// cctx.kernalContext().getSystemExecutorService().submit(() -> { -// try { -// for (BinaryMetadata meta : metas) -// procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); -// -// updateMetaFut.onDone(); -// } -// catch (Exception e) { -// updateMetaFut.onDone(e); -// } -// }); -// } -// else -// updateMetaFut.onDone(); -// -// for (String grpName : grpNames) { -// File cacheDir = resolveCacheDir(grpName); -// -// assert F.isEmpty(cacheDir.list()) : cacheDir; -// -// File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); -// -// if (!snapshotCacheDir.exists()) { -// log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); -// -// continue; -// } -// -// if (!cacheDir.exists()) { -// cacheDir.mkdir(); -// -// opCtx.cacheGroupFile(grpName, cacheDir); -// } -// -// try { -// for (File snpFile : snapshotCacheDir.listFiles()) { -// File target = new File(cacheDir, snpFile.getName()); -// -// log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); -// -// Files.copy(snpFile.toPath(), target.toPath()); -// -// opCtx.cacheGroupFile(grpName, target); -// } -// } -// catch (IOException e) { -// throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); -// } -// } -// -// return updateMetaFut; -// } - protected void rollbackRestoreOperation(Collection files) { List dirs = new ArrayList<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index bb417cfcfa0a0f..d826646636b882 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -79,13 +79,13 @@ public class SnapshotRestoreCacheGroupProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess prepareRestoreProc; + private final DistributedProcess prepareRestoreProc; /** Cache group restore perform phase. */ - private final DistributedProcess performRestoreProc; + private final DistributedProcess performRestoreProc; /** Cache group restore rollback phase. */ - private final DistributedProcess rollbackRestoreProc; + private final DistributedProcess rollbackRestoreProc; /** Logger. */ private final IgniteLogger log; @@ -96,6 +96,7 @@ public class SnapshotRestoreCacheGroupProcess { /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); + /** Stopped flag. */ private volatile boolean stopped; /** @@ -140,7 +141,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - SnapshotRestoreRequest req = new SnapshotRestoreRequest(snpName, cacheGrpNames, srvNodeIds); + SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, cacheGrpNames, srvNodeIds); fut = new RestoreSnapshotFuture(true); @@ -239,7 +240,7 @@ public void onNodeLeft(UUID leftNodeId) { if (fut0.isDone()) return; - SnapshotRestoreRequest req = fut0.request(); + SnapshotRestorePrepareRequest req = fut0.request(); if (req != null && req.requiredNodes().contains(leftNodeId)) { fut.handleError(new IgniteException(OP_REJECT_MSG + @@ -263,7 +264,7 @@ public void stop(String reason) { if (fut0.isDone()) return; - SnapshotRestoreRequest req = fut0.request(); + SnapshotRestorePrepareRequest req = fut0.request(); if (req == null) return; @@ -285,7 +286,7 @@ public void start() { * @param req Snapshot restore request. * @return Result future. */ - private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { + private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -385,7 +386,19 @@ private void finishPrepare(UUID reqId, Map return; } - List notFoundGroups = new ArrayList<>(fut.request().groups()); + UUID updateMetadataNode = null; + + for (Map.Entry entry : res.entrySet()) { + SnapshotRestorePrepareResponse resp = entry.getValue(); + + if (!F.isEmpty(resp.groups())) { + updateMetadataNode = entry.getKey(); + + break; + } + } + + List notFoundGroups = new ArrayList<>(fut0.request().groups()); try { Collection grpsDetails = mergeNodeResults(res); @@ -408,14 +421,14 @@ private void finishPrepare(UUID reqId, Map notFoundGroups.remove(grpDetails.groupName()); - PendingStartCacheGroup pendingGrp = fut.pendingStartCaches.get(grpDetails.groupName()); + PendingStartCacheGroup pendingGrp = fut0.pendingStartCaches.get(grpDetails.groupName()); for (StoredCacheData cacheData : grpDetails.configs()) { String cacheName = cacheData.config().getName(); // todo replace shared with groupName if (grpDetails.shared()) { - fut.addCacheId(CU.cacheId(cacheName)); + fut0.addCacheId(CU.cacheId(cacheName)); pendingGrp.caches.add(cacheName); } @@ -432,14 +445,14 @@ private void finishPrepare(UUID reqId, Map if (!notFoundGroups.isEmpty()) { throw new IllegalArgumentException("Cache group(s) \"" + F.concat(notFoundGroups, ", ") + - "\" not found in snapshot \"" + fut.request().snapshotName() + "\""); + "\" not found in snapshot \"" + fut0.request().snapshotName() + "\""); } Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - Set reqNodes = new HashSet<>(fut.request().requiredNodes()); + Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); reqNodes.removeAll(srvNodeIds); @@ -448,16 +461,25 @@ private void finishPrepare(UUID reqId, Map "the cluster [nodeIds=" + F.concat(reqNodes, ", ") + ']'); } - fut.startConfigs(cacheCfgs); + fut0.startConfigs(cacheCfgs); } catch (Exception e) { - fut.onDone(e); + fut0.onDone(e); return; } - if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut.isDone()) - performRestoreProc.start(reqId, fut.request()); + SnapshotRestorePrepareRequest req = fut0.request(); + + if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) + performRestoreProc.start(reqId, + new SnapshotRestorePerformRequest( + req.requestId(), + req.snapshotName(), + req.groups(), + req.requiredNodes(), + updateMetadataNode) + ); } private Collection mergeNodeResults(Map responses) { @@ -496,11 +518,15 @@ private Collection mergeNodeResults(Map perform(SnapshotRestoreRequest req) { + private IgniteInternalFuture perform(SnapshotRestorePerformRequest req) { if (ctx.clientNode() || !req.requiredNodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); - if (!req.equals(fut.request())) + RestoreSnapshotFuture fut0 = fut; + + SnapshotRestorePrepareRequest req0 = fut0.request(); + + if (req0 == null || !req.requestId().equals(req0.requestId())) return errResponse("Unknown snapshot restore operation was rejected."); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -508,7 +534,7 @@ private IgniteInternalFuture perform(SnapshotRes // todo check that snapshot exists ctx.getSystemExecutorService().submit(() -> { try { - performRestore(req, fut.rollbackContext()); + performRestore(req, fut0.rollbackContext()); retFut.onDone(); } catch (Throwable t) { @@ -519,11 +545,11 @@ private IgniteInternalFuture perform(SnapshotRes return retFut; } - private void performRestore(SnapshotRestoreRequest req, RestoreOperationContext opCtx) throws IgniteCheckedException { + private void performRestore(SnapshotRestorePerformRequest req, RestoreOperationContext opCtx) throws IgniteCheckedException { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); -// if (req.ctx.localNodeId() && !stopped) - snapshotMgr.registerSnapshotMetadata(req.snapshotName()); + if (ctx.localNodeId().equals(req.updateMetaNodeId()) && !stopped) + snapshotMgr.registerSnapshotMetadata(req.snapshotName()); for (String grpName : req.groups()) { rollbackLock.lock(); @@ -571,7 +597,7 @@ private void finishPerform(UUID reqId, Map if (U.isLocalNodeCoordinator(ctx.discovery())) { log.info("Starting snapshot restore rollback routine."); - rollbackRestoreProc.start(reqId, fut0.request()); + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(fut0.request().requestId(), failure)); } return; @@ -582,19 +608,22 @@ private void finishPerform(UUID reqId, Map } // todo separate rollback request - private IgniteInternalFuture rollback(SnapshotRestoreRequest req) { + private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); RestoreSnapshotFuture fut0 = fut; - // - if (!req.equals(fut0.request())) - return errResponse("Unknown snapshot restore operation was rejected [fut=" + fut + ", req=" + req + ']'); + SnapshotRestorePrepareRequest req0 = fut0.request(); - for (String grpName : req.groups()) + if (req0 == null || !req.requestId().equals(req0.requestId())) + return errResponse("Unknown snapshot restore rollback operation was rejected [fut=" + fut + ", req=" + req + ']'); + + for (String grpName : req0.groups()) rollbackChanges(fut0, grpName); + fut0.handleError(req.reason()); + return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse()); } @@ -673,9 +702,8 @@ protected static class RestoreSnapshotFuture extends GridFutureAdapter { /** Request ID. */ private final boolean initiator; - private final AtomicReference reqRef = new AtomicReference<>(); + private final AtomicReference reqRef = new AtomicReference<>(); -// private volatile RestoreGroupContext rollbackCtx; private final RestoreOperationContext rollbackCtx = new RestoreOperationContext(); private volatile Throwable err; @@ -686,10 +714,9 @@ public Throwable failure() { private volatile Collection cacheCfgsToStart; -// private final Set pendingStartCaches = new GridConcurrentHashSet<>(); - Map pendingStartCaches = new ConcurrentHashMap<>(); + private final Map pendingStartCaches = new ConcurrentHashMap<>(); - public SnapshotRestoreRequest request() { + public SnapshotRestorePrepareRequest request() { return reqRef.get(); } @@ -703,12 +730,10 @@ public void addCacheId(int cacheId) { cacheIds.add(cacheId); } - public boolean request(SnapshotRestoreRequest req) { + public boolean request(SnapshotRestorePrepareRequest req) { if (!reqRef.compareAndSet(null, req)) return false; -// pendingStartCaches.addAll(req.groups()); - for (String grpName : req.groups()) { cacheIds.add(CU.cacheId(grpName)); @@ -731,7 +756,7 @@ public boolean initiator() { /** @return Request ID. */ public UUID id() { - SnapshotRestoreRequest req = reqRef.get(); + SnapshotRestorePrepareRequest req = reqRef.get(); return req != null ? req.requestId() : null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java index d6163e0b274b9f..ace3983e109992 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java @@ -1,22 +1,39 @@ +/* + * 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.Set; import java.util.UUID; -public class SnapshotRestorePerformRequest extends SnapshotRestoreRequest { +public class SnapshotRestorePerformRequest extends SnapshotRestorePrepareRequest { /** Serial version uid. */ private static final long serialVersionUID = 0L; private final UUID updateMetaNodeId; - public SnapshotRestorePerformRequest(String snpName, Collection grps, Set reqNodes, UUID updateMetaNodeId) { - super(snpName, grps, reqNodes); + public SnapshotRestorePerformRequest(UUID reqId, String snpName, Collection grps, Set reqNodes, UUID updateMetaNodeId) { + super(reqId, snpName, grps, reqNodes); this.updateMetaNodeId = updateMetaNodeId; } - public UUID getUpdateMetaNodeId() { + public UUID updateMetaNodeId() { return updateMetaNodeId; } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java index 4c4e5da4f5d7ed..df887d9fbfda6b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java @@ -20,7 +20,7 @@ import java.io.Serializable; /** - * Snapshot restore operation single node response. + * Snapshot restore perform operation single node response. */ public class SnapshotRestorePerformResponse implements Serializable { /** Serial version uid. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java similarity index 85% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index 61c15cad404f73..7fa855208690d8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; -public class SnapshotRestoreRequest implements Serializable { +public class SnapshotRestorePrepareRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -41,12 +41,13 @@ public class SnapshotRestoreRequest implements Serializable { private final Set reqNodes; /** Request ID. */ - private final UUID reqId = UUID.randomUUID(); + private final UUID reqId; - public SnapshotRestoreRequest(String snpName, Collection grps, Set reqNodes) { + public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection grps, Set reqNodes) { this.snpName = snpName; this.grps = grps; this.reqNodes = reqNodes; + this.reqId = reqId; } public UUID requestId() { @@ -73,7 +74,7 @@ public Set requiredNodes() { if (o == null || getClass() != o.getClass()) return false; - return Objects.equals(reqId, ((SnapshotRestoreRequest)o).reqId); + return Objects.equals(reqId, ((SnapshotRestorePrepareRequest)o).reqId); } /** {@inheritDoc} */ @@ -83,6 +84,6 @@ public Set requiredNodes() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SnapshotRestoreRequest.class, this); + return S.toString(SnapshotRestorePrepareRequest.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java index 828203ef509a8b..80dbbb7b30c87d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -23,7 +23,7 @@ import org.apache.ignite.internal.processors.cache.StoredCacheData; /** - * Snapshot restore operation single node validation response. + * Snapshot restore prepare operation single node validation response. */ public class SnapshotRestorePrepareResponse implements Serializable { /** Serial version uid. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java new file mode 100644 index 00000000000000..9ce4e2d063ff24 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java @@ -0,0 +1,43 @@ +/* + * 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.UUID; + +public class SnapshotRestoreRollbackRequest implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + private final Throwable reason; + + private final UUID reqId; + + public SnapshotRestoreRollbackRequest(UUID reqId, Throwable reason) { + this.reqId = reqId; + this.reason = reason; + } + + public UUID requestId() { + return reqId; + } + + public Throwable reason() { + return reason; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java index 6153372308cba8..755d79e66bcc8c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java @@ -20,7 +20,7 @@ import java.io.Serializable; /** - * Snapshot restore operation single node response. + * Snapshot restore rollback operation single node response. */ public class SnapshotRestoreRollbackResponse implements Serializable { /** Serial version uid. */ From 8a9629b92838cc06f7c64af8b1a941773ae96430 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 21 Jan 2021 12:25:27 +0300 Subject: [PATCH 09/98] IGNITE-13805 Code cleanup. --- .../org/apache/ignite/IgniteSnapshot.java | 7 + .../processors/cache/ClusterCachesInfo.java | 10 +- .../cache/DynamicCacheChangeRequest.java | 7 + .../snapshot/CacheGroupSnapshotDetails.java | 69 ++++ .../snapshot/IgniteSnapshotManager.java | 234 ++++++++------ .../SnapshotRestoreCacheGroupProcess.java | 299 +++++++++--------- .../SnapshotRestorePerformRequest.java | 20 ++ .../SnapshotRestorePrepareRequest.java | 41 +-- .../SnapshotRestorePrepareResponse.java | 55 ---- .../SnapshotRestoreRollbackRequest.java | 21 ++ .../cluster/GridClusterStateProcessor.java | 6 - .../util/distributed/DistributedProcess.java | 15 +- .../IgniteClusterSnapshoRestoreSelfTest.java | 79 +++-- 13 files changed, 483 insertions(+), 380 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index a5f3110415d21c..3023c9ff0d10ac 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -50,5 +50,12 @@ public interface IgniteSnapshot { */ public IgniteFuture cancelSnapshot(String name); + /** + * Restore cache group(s) from the snapshot. + * + * @param snapshotName Snapshot name. + * @param cacheGroupNames Cache groups to be restored. + * @return Future which will be completed when restore operation finished. + */ public IgniteFuture restoreCacheGroups(String snapshotName, Collection cacheGroupNames); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index e20c29e2004b46..d074ff4d1e8c61 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1011,11 +1011,13 @@ else if (encMgr.masterKeyDigest() != null && if (err == null && !req.restoredCache()) { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - String conflictingName; + String conflictingName = cacheName; - if (snapshotMgr.isCacheRestoring(conflictingName = cacheName) || - ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheRestoring(conflictingName))) - err = new IgniteCheckedException("Cache start failed. A cache named \"" + conflictingName + "\" is currently being restored from a snapshot."); + if (snapshotMgr.isCacheRestoring(conflictingName) || + ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheRestoring(conflictingName))) { + err = new IgniteCheckedException("Cache start failed. A cache named \"" + conflictingName + + "\" is currently being restored from a snapshot."); + } } if (err != null) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java index f6817fe36ebff4..6293fb1dcba2b7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java @@ -107,6 +107,7 @@ public class DynamicCacheChangeRequest implements Serializable { /** Cache configuration enrichment. */ private CacheConfigurationEnrichment cacheCfgEnrichment; + /** Flag indicating that the cache was started after restoring from a snapshot. */ private boolean restoredCache; /** @@ -482,10 +483,16 @@ public void masterKeyDigest(@Nullable byte[] masterKeyDigest) { return masterKeyDigest; } + /** + * @param restoredCache Flag indicating that the cache was started after restoring from a snapshot. + */ public void restoredCache(boolean restoredCache) { this.restoredCache = restoredCache; } + /** + * @return Flag indicating that the cache was started after restoring from a snapshot. + */ public boolean restoredCache() { return restoredCache; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java new file mode 100644 index 00000000000000..2259d177f7443d --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java @@ -0,0 +1,69 @@ +/* + * 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.List; +import java.util.Set; +import org.apache.ignite.internal.processors.cache.StoredCacheData; + +/** */ +class CacheGroupSnapshotDetails implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Cache group name. */ + private String grpName; + + /** Local partition IDs. */ + private Set parts; + + /** Group cache configurations. */ + private List cfgs; + + /** + * @param cfgs Group cache configurations. + * @param parts Local partition IDs. + */ + public CacheGroupSnapshotDetails(String grpName, List cfgs, Set parts) { + this.grpName = grpName; + this.cfgs = cfgs; + this.parts = parts; + } + + /** + * @return Group cache configurations. + */ + public List configs() { + return cfgs; + } + + /** + * @return Local partition IDs. + */ + public Set parts() { + return parts; + } + + /** + * @return Cache group name. + */ + public String groupName() { + return grpName; + } +} 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 d7987a149b4172..399ab711fa5fad 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 @@ -84,6 +84,7 @@ import org.apache.ignite.internal.processors.cache.CacheType; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; +import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; @@ -135,6 +136,7 @@ import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.apache.ignite.thread.OomExceptionHandler; @@ -164,6 +166,7 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.pageIndex; import static org.apache.ignite.internal.pagemem.PageIdUtils.toDetailString; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; @@ -307,6 +310,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Last seen cluster snapshot operation. */ private volatile ClusterSnapshotFuture lastSeenSnpFut = new ClusterSnapshotFuture(); + /** Distributed process to restore cache group from the snapshot. */ private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProcess; /** @@ -467,7 +471,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onActivate(GridKernalContext kctx) { - restoreCacheGrpProcess.start(); + // No-op. } /** {@inheritDoc} */ @@ -772,10 +776,21 @@ public boolean isSnapshotCreating() { } } - public boolean isCacheRestoring(@Nullable String cacheName) { - return restoreCacheGrpProcess.inProgress(cacheName); + /** + * @param name Cache name or {@code null} to check if any cache is currently being restored. + * @return {@code True} if the restore operation is in progress. + */ + public boolean isCacheRestoring(@Nullable String name) { + return restoreCacheGrpProcess.inProgress(name); } + /** + * Callback from cache startup during cache group restore operation. + * + * @param cacheName Started cache name. + * @param grpName Started cache group name. + * @param err Error if any. + */ public void afterRestoredCacheStarted(String cacheName, @Nullable String grpName, @Nullable Throwable err) { restoreCacheGrpProcess.handleCacheStart(cacheName, grpName, err); } @@ -1018,6 +1033,11 @@ public List readSnapshotMetadatas(String snpName) { if (!clusterState.hasBaselineTopology()) throw new IgniteException("Snapshot operation has been rejected. The baseline topology is not configured for cluster."); + if (isCacheRestoring(null)) { + throw new IgniteException("Snapshot operation has been rejected. " + + "Cache group restore operation is currently in prgoress."); + } + if (cctx.kernalContext().clientNode()) { ClusterNode crd = U.oldest(cctx.kernalContext().discovery().aliveServerNodes(), null); @@ -1099,85 +1119,69 @@ public List readSnapshotMetadatas(String snpName) { return restoreCacheGrpProcess.start(snpName, grpNames); } - protected void checkMetaCompatibility(String snpName) throws IgniteCheckedException, IOException { - String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); - - File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); - - String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + - nodeFolderName; + /** + * @param snpName Snapshot name. + * @param checkCompatibility Don't update metadata, just check the compatibility of the snapshot metadata. + * @param failIfAbsent Throw an exception if the snapshot metadata folder doesn't exists. + * @throws IgniteCheckedException If failed. + */ + protected void mergeSnapshotMetadata( + String snpName, + boolean checkCompatibility, + boolean failIfAbsent + ) throws IgniteCheckedException { + File snpMetaDir = new File(snapshotLocalDir(snpName), + DFLT_BINARY_METADATA_PATH + File.separator + pdsSettings.folderName()); - File snapshotMetadataDir = new File(workDIr, subPath); + if (!snpMetaDir.exists()) { + if (failIfAbsent) { + throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + + "directory doesn't exists [snpName=" + snpName + ", dir=" + snpMetaDir + ']'); + } - if (!snapshotMetadataDir.exists()) return; + } - // todo get binaryContext without cast + Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); + ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); - for (File file : snapshotMetadataDir.listFiles()) { + for (File file : snpMetaDir.listFiles()) { try (FileInputStream in = new FileInputStream(file)) { - BinaryMetadata meta = U.unmarshal(cctx.kernalContext().config().getMarshaller(), in, U.resolveClassLoader(cctx.kernalContext().config())); + BinaryMetadata newMeta = U.unmarshal(marshaller, in, clsLdr); - BinaryMetadata oldMeta = binProc.metadata0(meta.typeId()); + if (!checkCompatibility) { + binProc.addMeta(newMeta.typeId(), newMeta.wrap(binProc.binaryContext()), false); - if (oldMeta == null) continue; + } - BinaryUtils.mergeMetadata(oldMeta, meta, null); - } - } - } - - protected void registerSnapshotMetadata(String snpName) throws IgniteCheckedException { - String nodeFolderName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); - - File workDIr = resolveSnapshotWorkDirectory(cctx.kernalContext().config()); - - String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + "binary_meta" + File.separator + - nodeFolderName; - - File snapshotMetadataDir = new File(workDIr, subPath); - - // todo remove if executed from single node - if (!snapshotMetadataDir.exists()) - return; - - // restore metadata - CacheObjectBinaryProcessorImpl procImpl = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); - - List metas = new ArrayList<>(); + BinaryMetadata oldMeta = binProc.binaryMetadata(newMeta.typeId()); - Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); - ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); + if (oldMeta == null) + continue; - for (File file : snapshotMetadataDir.listFiles()) { - try (FileInputStream in = new FileInputStream(file)) { - metas.add(U.unmarshal(marshaller, in, clsLdr)); + BinaryUtils.mergeMetadata(oldMeta, newMeta, null); } - catch (Exception e) { - throw new IgniteCheckedException("Failed to add metadata from file: " + file.getName() + - "; exception was thrown: " + e.getMessage()); + catch (IOException e) { + throw new IgniteCheckedException("Failed to read metadata " + file, e); } } - - // todo should register only from one node and validate result - for (BinaryMetadata meta : metas) - procImpl.addMeta(meta.typeId(), meta.wrap(procImpl.binaryContext()), false); } + /** + * @param snpName Snapshot name. + * @param grpName Cache group name. + * @param newFiles A list to keep track of the files created, the list updates during the restore process. + * @throws IgniteCheckedException If failed. + */ protected void restoreCacheGroupFiles(String snpName, String grpName, List newFiles) throws IgniteCheckedException { - File cacheDir = resolveCacheDir(grpName); - - assert F.isEmpty(cacheDir.list()) : cacheDir; - - File snapshotCacheDir = resolveSnapshotCacheDir(snpName, cctx.kernalContext().config(), grpName); - - if (!snapshotCacheDir.exists()) { - log.info("Skipping restore of cache group [snapshot=" + snpName + ", cache=" + grpName + "]"); + File snapshotCacheDir = resolveCacheDir(snapshotLocalDir(snpName), grpName); + if (!snapshotCacheDir.exists()) return; - } + + File cacheDir = resolveCacheDir(new File(cctx.kernalContext().config().getWorkDirectory()), grpName); if (!cacheDir.exists()) { cacheDir.mkdir(); @@ -1185,14 +1189,24 @@ protected void restoreCacheGroupFiles(String snpName, String grpName, List newFiles.add(cacheDir); } else - if (cacheDir.list().length != 0) - throw new IgniteCheckedException("Unable to restore cache group, directory is not empty [group=" + grpName + ", dir=" + cacheDir + ']'); + if (cacheDir.list().length > 0) { + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } try { + if (log.isInfoEnabled()) { + log.info("Copying partition files of the cache group " + + "[from=" + snapshotCacheDir + ", to=" + cacheDir + ']'); + } + for (File snpFile : snapshotCacheDir.listFiles()) { File target = new File(cacheDir, snpFile.getName()); - log.info("Restore file from snapshot [snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); + if (log.isDebugEnabled()) { + log.debug("Restoring partition file from the snapshot [snapshot=" + snpName + + ", grp=" + grpName + ", src=" + snpFile + ", target=" + target + "]"); + } newFiles.add(target); @@ -1200,10 +1214,13 @@ protected void restoreCacheGroupFiles(String snpName, String grpName, List } } catch (IOException e) { - throw new IgniteCheckedException("Unable to restore file [snapshot=" + snpName + ", grp=" + grpName + ']', e); + throw new IgniteCheckedException("Unable to copy file [snapshot=" + snpName + ", grp=" + grpName + ']', e); } } + /** + * @param files List pf created files. + */ protected void rollbackRestoreOperation(Collection files) { List dirs = new ArrayList<>(); @@ -1221,32 +1238,65 @@ protected void rollbackRestoreOperation(Collection files) { dir.delete(); } - private File resolveCacheDir(String cacheOrGrpName) throws IgniteCheckedException { - File workDIr = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); - - String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName() + File.separator; + /** + * @param workDir Work directory. + * @param cacheName Cache (group) name. + * @return Local path to the cache directory. + */ + private File resolveCacheDir(File workDir, String cacheName) { + String dbPath = DFLT_STORE_DIR + File.separator + pdsSettings.folderName() + File.separator; - File cacheDir = new File(workDIr, nodeDirName + CACHE_DIR_PREFIX + cacheOrGrpName); + File cacheDir = new File(workDir, dbPath + CACHE_DIR_PREFIX + cacheName); if (cacheDir.exists()) return cacheDir; - return new File(workDIr, nodeDirName + CACHE_GRP_DIR_PREFIX + cacheOrGrpName); + return new File(workDir, dbPath + CACHE_GRP_DIR_PREFIX + cacheName); } - protected File resolveSnapshotCacheDir(String snpName, IgniteConfiguration cfg, String cacheName) throws IgniteCheckedException { - File workDIr = resolveSnapshotWorkDirectory(cfg); + /** + * @param snpName Snapshot name. + * @param grpName Cache group name. + * @return Details about the locally stored cache group, or {@code null} if cache group (or snapshot) was not found. + * @throws IgniteCheckedException if failed. + */ + protected @Nullable CacheGroupSnapshotDetails readCacheGroupDetails( + String snpName, + String grpName + ) throws IgniteCheckedException { + IgniteConfiguration nodeCfg = cctx.kernalContext().config(); + File cacheDir = resolveCacheDir(snapshotLocalDir(snpName), grpName); - String nodeDirName = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName(); + if (!cacheDir.exists()) + return null; - String subPath = snpName + File.separator + DFLT_STORE_DIR + File.separator + nodeDirName + File.separator; + List cacheCfgs = new ArrayList<>(1); + Set parts = new HashSet<>(); - File cacheDir = new File(workDIr, subPath + CACHE_DIR_PREFIX + cacheName); + JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(nodeCfg.getIgniteInstanceName()); + ClassLoader clsLdr = U.resolveClassLoader(nodeCfg); - if (cacheDir.exists()) - return cacheDir; + for (File file : cacheDir.listFiles()) { + if (file.isDirectory()) + continue; + + String name = file.getName(); + + if (name.endsWith(CACHE_DATA_FILENAME) && file.length() > 0) { + try (InputStream stream = new BufferedInputStream(new FileInputStream(file))) { + cacheCfgs.add(marshaller.unmarshal(stream, clsLdr)); + } catch (IOException e) { + throw new IgniteCheckedException("Unable to read stored cache configuration: " + e.getMessage(), e); + } + } + else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { + String partId = name.substring(FilePageStoreManager.PART_FILE_PREFIX.length(), name.indexOf('.')); - return new File(workDIr, subPath + CACHE_GRP_DIR_PREFIX + cacheName); + parts.add(Integer.parseInt(partId)); + } + } + + return new CacheGroupSnapshotDetails(grpName, cacheCfgs, parts); } /** {@inheritDoc} */ @@ -2172,30 +2222,4 @@ public IgniteSnapshotFutureImpl(IgniteInternalFuture fut) { return new IgniteException("Snapshot has not been created", U.convertException(e)); } } - -// static class RestoreGroupContext { -// private final List files = new ArrayList<>(); -// -// public void addFile(File file) { -// files.add(file); -// } -// -// public List files() { -// return files; -// } -// -//// private final List metadataTypes = new ArrayList<>(); -//// -//// public void cacheGroupFile(String grpName, File newFile) { -//// files.computeIfAbsent(grpName, v -> new ArrayList<>()).add(newFile); -//// } -//// -//// public void metadataType(int typeId) { -//// metadataTypes.add(typeId); -//// } -//// -//// public List cacheGroupFiles(String grpName) { -//// return files.get(grpName); -//// } -// } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index d826646636b882..4d5181676e04b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,11 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.io.BufferedInputStream; import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.io.InputStream; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -31,20 +27,17 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestorePrepareResponse.CacheGroupSnapshotDetails; +import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -58,22 +51,18 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteFuture; -import org.apache.ignite.marshaller.MarshallerUtils; -import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.END_SNAPSHOT_RESTORE; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT_RESTORE; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.UNDO_SNAPSHOT_RESTORE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; +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_ROLLBACK; /** * Distributed process to restore cache group from the snapshot. */ public class SnapshotRestoreCacheGroupProcess { /** Reject operation message. */ - private static final String OP_REJECT_MSG = "Snapshot restore operation was rejected. "; + private static final String OP_REJECT_MSG = "Cache group restore operation was rejected. "; /** Kernal context. */ private final GridKernalContext ctx; @@ -90,12 +79,12 @@ public class SnapshotRestoreCacheGroupProcess { /** Logger. */ private final IgniteLogger log; + /** The future to be completed when the cache restore process is complete. */ + private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(); + /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); - /** The future to be completed when the cache restore process is complete. */ - private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(false); - /** Stopped flag. */ private volatile boolean stopped; @@ -107,9 +96,12 @@ public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { log = ctx.log(getClass()); - prepareRestoreProc = new DistributedProcess<>(ctx, START_SNAPSHOT_RESTORE, this::prepare, this::finishPrepare); - performRestoreProc = new DistributedProcess<>(ctx, END_SNAPSHOT_RESTORE, this::perform, this::finishPerform); - rollbackRestoreProc = new DistributedProcess<>(ctx, UNDO_SNAPSHOT_RESTORE, this::rollback, this::finishRollback); + prepareRestoreProc = + new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, this::prepare, this::finishPrepare); + performRestoreProc = + new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, this::perform, this::finishPerform); + rollbackRestoreProc = + new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK, this::rollback, this::finishRollback); fut.onDone(); } @@ -134,16 +126,28 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "The previous snapshot restore operation was not completed.")); } - if (!ctx.state().clusterState().state().active()) + DiscoveryDataClusterState clusterState = ctx.state().clusterState(); + + if (!clusterState.state().active()) return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); + if (!clusterState.hasBaselineTopology()) { + return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + + "The baseline topology is not configured for cluster.")); + } + + if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) { + return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + + "A cluster snapshot operation is in progress.")); + } + Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - F.node2id(), - (node) -> CU.baselineNode(node, ctx.state().clusterState()))); + F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, cacheGrpNames, srvNodeIds); + SnapshotRestorePrepareRequest req = + new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, cacheGrpNames, srvNodeIds); - fut = new RestoreSnapshotFuture(true); + fut = new RestoreSnapshotFuture(); prepareRestoreProc.start(req.requestId(), req); @@ -277,100 +281,91 @@ public void stop(String reason) { fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + reason)); } - /** */ - public void start() { - stopped = false; - } - /** - * @param req Snapshot restore request. + * Cache group napshot restore single node validation phase. + * + * @param req Request to prepare snapshot restore. * @return Result future. */ private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); + stopped = false; + if (inProgress(null)) return errResponse(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); if (!ctx.state().clusterState().state().active()) return errResponse(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); + // Skip creating future on initiator. if (fut.isDone()) - fut = new RestoreSnapshotFuture(false); + fut = new RestoreSnapshotFuture(); fut.request(req); - List grpCfgs = new ArrayList<>(); + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) + return new GridFinishedFuture<>(); - // Collect cache configuration(s). - for (String cacheName : req.groups()) { + GridFutureAdapter retFut = new GridFutureAdapter<>(); + + ctx.getSystemExecutorService().submit(() -> { try { - CacheGroupSnapshotDetails grpCfg = readCacheGroupDetails(req.snapshotName(), cacheName); + SnapshotRestorePrepareResponse res = prepare0(req); - if (grpCfg != null) - grpCfgs.add(grpCfg); + retFut.onDone(res); } - catch (IOException | IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - } - - if (grpCfgs.isEmpty()) - return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); - - try { - ctx.cache().context().snapshotMgr().checkMetaCompatibility(req.snapshotName()); - } - catch (BinaryObjectException e) { - log.warning(OP_REJECT_MSG + "Incompatible binary types found", e); + catch (BinaryObjectException e) { + log.warning(OP_REJECT_MSG + "Incompatible binary types found", e); - return errResponse(OP_REJECT_MSG + "Incompatible binary types found: " + e.getMessage()); - } - catch (IOException | IgniteCheckedException e) { - return errResponse(new IgniteException("Prepare phase has failed: " + e.getMessage(), e)); - } + retFut.onDone(new IgniteException(OP_REJECT_MSG + "Incompatible binary types found: " + e.getMessage())); + } + catch (Throwable t) { + retFut.onDone(t); + } + }); - return new GridFinishedFuture<>(new SnapshotRestorePrepareResponse(grpCfgs)); + return retFut; } /** - * @param snapshotName Snapshot name. - * @param grpName Cache group name. - * @return Details about the locally stored cache group, or {@code null} if cache group (or snapshot) was not found. - * @throws IgniteCheckedException if failed. - * @throws IOException if I/O errors occur during reading cache configurations. + * Reads locally stored cache configurations and verifies that the binary metadata can be merged from the snapshot. + * + * @param req Request to prepare snapshot restore. + * @return Response to prepare snapshot restore. + * @throws IgniteCheckedException If failed. */ - private @Nullable CacheGroupSnapshotDetails readCacheGroupDetails(String snapshotName, String grpName) throws IgniteCheckedException, IOException { - File cacheDir = ctx.cache().context().snapshotMgr().resolveSnapshotCacheDir(snapshotName, ctx.config(), grpName); - - if (!cacheDir.exists()) - return null; - - Set parts = new HashSet<>(); - - List cacheCfgs = new ArrayList<>(1); - - for (File file : cacheDir.listFiles()) { - if (file.isDirectory()) - continue; + private @Nullable SnapshotRestorePrepareResponse prepare0( + SnapshotRestorePrepareRequest req + ) throws IgniteCheckedException { + List grpCfgs = new ArrayList<>(); - String name = file.getName(); + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - if (name.endsWith(CACHE_DATA_FILENAME) && file.length() > 0) - cacheCfgs.add(unmarshal(ctx.config(), file)); - else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { - String partId = name.substring(FilePageStoreManager.PART_FILE_PREFIX.length(), name.indexOf('.')); + // Collect cache configuration(s). + for (String cacheName : req.groups()) { + CacheGroupSnapshotDetails grpCfg = snapshotMgr.readCacheGroupDetails(req.snapshotName(), cacheName); - parts.add(Integer.parseInt(partId)); - } + if (grpCfg != null) + grpCfgs.add(grpCfg); } - boolean sharedGrp = cacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX); + if (grpCfgs.isEmpty()) + return null; - return new CacheGroupSnapshotDetails(grpName, sharedGrp, cacheCfgs, parts); + ctx.cache().context().snapshotMgr().mergeSnapshotMetadata(req.snapshotName(), true, false); + + return new SnapshotRestorePrepareResponse(grpCfgs); } + /** + * Completes the verification phase and starts the restore performing phase if there were no errors. + * + * @param reqId Request ID. + * @param res Results. + * @param errs Errors. + */ private void finishPrepare(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; @@ -391,7 +386,7 @@ private void finishPrepare(UUID reqId, Map for (Map.Entry entry : res.entrySet()) { SnapshotRestorePrepareResponse resp = entry.getValue(); - if (!F.isEmpty(resp.groups())) { + if (resp != null && !F.isEmpty(resp.groups())) { updateMetadataNode = entry.getKey(); break; @@ -426,8 +421,7 @@ private void finishPrepare(UUID reqId, Map for (StoredCacheData cacheData : grpDetails.configs()) { String cacheName = cacheData.config().getName(); - // todo replace shared with groupName - if (grpDetails.shared()) { + if (!F.isEmpty(cacheData.config().getGroupName())) { fut0.addCacheId(CU.cacheId(cacheName)); pendingGrp.caches.add(cacheName); } @@ -482,13 +476,20 @@ private void finishPrepare(UUID reqId, Map ); } - private Collection mergeNodeResults(Map responses) { + /** + * @param res Results from multiple nodes. + * @return A collection that contains information about the snapshot cache group(s) on all nodes. + */ + private Collection mergeNodeResults(Map res) { Map> globalDetails = new HashMap<>(); - for (Map.Entry entry : responses.entrySet()) { + for (Map.Entry entry : res.entrySet()) { UUID currNodeId = entry.getKey(); SnapshotRestorePrepareResponse singleResp = entry.getValue(); + if (singleResp == null) + continue; + for (CacheGroupSnapshotDetails nodeDetails : singleResp.groups()) { T2 clusterDetailsPair = globalDetails.get(nodeDetails.groupName()); @@ -518,6 +519,10 @@ private Collection mergeNodeResults(Map perform(SnapshotRestorePerformRequest req) { if (ctx.clientNode() || !req.requiredNodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); @@ -531,10 +536,12 @@ private IgniteInternalFuture perform(SnapshotRes GridFutureAdapter retFut = new GridFutureAdapter<>(); - // todo check that snapshot exists + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) + return new GridFinishedFuture<>(); + ctx.getSystemExecutorService().submit(() -> { try { - performRestore(req, fut0.rollbackContext()); + restore0(req, fut0.rollbackContext()); retFut.onDone(); } catch (Throwable t) { @@ -545,11 +552,17 @@ private IgniteInternalFuture perform(SnapshotRes return retFut; } - private void performRestore(SnapshotRestorePerformRequest req, RestoreOperationContext opCtx) throws IgniteCheckedException { + /** + * @param req Request to perform snapshot restore. + * @param opCtx Restore operation context. + * @throws IgniteCheckedException If failed. + */ + private void restore0(SnapshotRestorePerformRequest req, + RestoreOperationContext opCtx) throws IgniteCheckedException { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); if (ctx.localNodeId().equals(req.updateMetaNodeId()) && !stopped) - snapshotMgr.registerSnapshotMetadata(req.snapshotName()); + snapshotMgr.mergeSnapshotMetadata(req.snapshotName(), false, true); for (String grpName : req.groups()) { rollbackLock.lock(); @@ -563,18 +576,19 @@ private void performRestore(SnapshotRestorePerformRequest req, RestoreOperationC opCtx.put(grpName, newFiles); snapshotMgr.restoreCacheGroupFiles(req.snapshotName(), grpName, newFiles); - } catch (IgniteCheckedException e) { - RestoreSnapshotFuture fut0 = fut; - - if (fut0 != null && fut0.id().equals(req.requestId())) - fut0.onDone(e); - } finally { + } + finally { rollbackLock.unlock(); } } } - private void finishPerform(UUID reqId, Map map, Map errs) { + /** + * @param reqId Request ID. + * @param res Results. + * @param errs Errors. + */ + private void finishPerform(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; Throwable failure = F.first(errs.values()); @@ -582,12 +596,13 @@ private void finishPerform(UUID reqId, Map if (failure == null) failure = fut0.failure(); - if (failure == null && !map.keySet().containsAll(fut0.request().requiredNodes())) { + if (failure == null && !res.keySet().containsAll(fut0.request().requiredNodes())) { Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); - reqNodes.removeAll(map.keySet()); + reqNodes.removeAll(res.keySet()); - log.warning("Node left the cluster, snapshot restore operation should be reverted [nodeIds=" + F.concat(reqNodes, ", ")); + log.warning("Node(s) left the cluster, " + + "snapshot restore operation should be reverted [nodeIds=" + F.concat(reqNodes, ", ")); fut0.handleError(failure = new IgniteException(new IgniteException(OP_REJECT_MSG + "Baseline node has left the cluster [nodeId(s)=" + F.concat(reqNodes, ", ") + ']'))); @@ -607,7 +622,10 @@ private void finishPerform(UUID reqId, Map ctx.cache().dynamicStartCachesByStoredConf(fut0.startConfigs(), true, true, false, null, true); } - // todo separate rollback request + /** + * @param req Request to rollback snapshot restore. + * @return Result future. + */ private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -617,7 +635,7 @@ private IgniteInternalFuture rollback(SnapshotR SnapshotRestorePrepareRequest req0 = fut0.request(); if (req0 == null || !req.requestId().equals(req0.requestId())) - return errResponse("Unknown snapshot restore rollback operation was rejected [fut=" + fut + ", req=" + req + ']'); + return errResponse("Unknown snapshot restore rollback operation was rejected."); for (String grpName : req0.groups()) rollbackChanges(fut0, grpName); @@ -627,7 +645,12 @@ private IgniteInternalFuture rollback(SnapshotR return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse()); } - private void finishRollback(UUID reqId, Map map, Map errs) { + /** + * @param reqId Request ID. + * @param res Results. + * @param errs Errors. + */ + private void finishRollback(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; if (!F.isEmpty(errs)) { @@ -646,38 +669,39 @@ private void finishRollback(UUID reqId, Map err, RestoreSnapshotFuture fut) { - if (!fut.id().equals(reqId) || fut.isDone()) + if (!reqId.equals(fut.id()) || fut.isDone()) return false; return !F.isEmpty(err) ? fut.onDone(F.firstValue(err)) : fut.onDone(); } - private StoredCacheData unmarshal(IgniteConfiguration cfg, File cacheDataFile) throws IOException, IgniteCheckedException { - JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(cfg.getIgniteInstanceName()); - - try (InputStream stream = new BufferedInputStream(new FileInputStream(cacheDataFile))) { - StoredCacheData data = marshaller.unmarshal(stream, U.resolveClassLoader(cfg)); - - return data; - } - } - + /** + * @param msg Error message. + * @param Type of the future. + * @return Failed with the specified error message future. + */ private IgniteInternalFuture errResponse(String msg) { return errResponse(new IgniteException(msg)); } + /** + * @param ex Exception. + * @param Type of the future. + * @return Failed with the specified exception future. + */ private IgniteInternalFuture errResponse(Exception ex) { - //return errResponse(msg, null); return new GridFinishedFuture<>(ex); } - static class PendingStartCacheGroup { + /** */ + private static class PendingStartCacheGroup { volatile boolean canRollback = true; Set caches = new GridConcurrentHashSet<>(); } - static class RestoreOperationContext { + /** */ + private static class RestoreOperationContext { private final Map> newGrpFiles = new HashMap<>(); public List get(String grpName) { @@ -698,16 +722,13 @@ public void put(String grpName, List files) { } /** */ - protected static class RestoreSnapshotFuture extends GridFutureAdapter { - /** Request ID. */ - private final boolean initiator; - - private final AtomicReference reqRef = new AtomicReference<>(); - + private static class RestoreSnapshotFuture extends GridFutureAdapter { private final RestoreOperationContext rollbackCtx = new RestoreOperationContext(); private volatile Throwable err; + private volatile SnapshotRestorePrepareRequest req; + public Throwable failure() { return err; } @@ -717,7 +738,7 @@ public Throwable failure() { private final Map pendingStartCaches = new ConcurrentHashMap<>(); public SnapshotRestorePrepareRequest request() { - return reqRef.get(); + return req; } public Set cacheIds = new GridConcurrentHashSet<>(); @@ -730,34 +751,18 @@ public void addCacheId(int cacheId) { cacheIds.add(cacheId); } - public boolean request(SnapshotRestorePrepareRequest req) { - if (!reqRef.compareAndSet(null, req)) - return false; + public void request(SnapshotRestorePrepareRequest req) { + this.req = req; for (String grpName : req.groups()) { cacheIds.add(CU.cacheId(grpName)); pendingStartCaches.put(grpName, new PendingStartCacheGroup()); } - - return true; - } - - /** - * @param initiator A flag indicating that the node is the initiator of the request. - */ - RestoreSnapshotFuture(boolean initiator) { - this.initiator = initiator; - } - - public boolean initiator() { - return initiator; } /** @return Request ID. */ public UUID id() { - SnapshotRestorePrepareRequest req = reqRef.get(); - return req != null ? req.requestId() : null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java index ace3983e109992..c42e17f680b34d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java @@ -20,20 +20,40 @@ import java.util.Collection; import java.util.Set; import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +/** + * Request to perform snapshot restore. + */ public class SnapshotRestorePerformRequest extends SnapshotRestorePrepareRequest { /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** Node ID from which to update the binary metadata. */ private final UUID updateMetaNodeId; + /** + * @param reqId Request ID. + * @param snpName Snapshot name. + * @param grps List of cache group names to restore from the snapshot. + * @param reqNodes List of baseline node IDs that must be alive to complete the operation. + * @param updateMetaNodeId Node ID from which to update the binary metadata. + */ public SnapshotRestorePerformRequest(UUID reqId, String snpName, Collection grps, Set reqNodes, UUID updateMetaNodeId) { super(reqId, snpName, grps, reqNodes); this.updateMetaNodeId = updateMetaNodeId; } + /** + * @return Node ID from which to update the binary metadata. + */ public UUID updateMetaNodeId() { return updateMetaNodeId; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestorePerformRequest.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index 7fa855208690d8..57a68aabb83fca 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -20,12 +20,14 @@ import java.io.Serializable; import java.util.Collection; import java.util.Collections; -import java.util.Objects; import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; +/** + * Request to prepare snapshot restore. + */ public class SnapshotRestorePrepareRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -33,16 +35,23 @@ public class SnapshotRestorePrepareRequest implements Serializable { /** Snapshot name. */ private final String snpName; - /** The list of cache groups to restore from the snapshot. */ + /** List of cache group names to restore from the snapshot. */ @GridToStringInclude private final Collection grps; + /** List of baseline node IDs that must be alive to complete the operation. */ @GridToStringInclude private final Set reqNodes; /** Request ID. */ private final UUID reqId; + /** + * @param reqId Request ID. + * @param snpName Snapshot name. + * @param grps List of cache group names to restore from the snapshot. + * @param reqNodes List of baseline node IDs that must be alive to complete the operation. + */ public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection grps, Set reqNodes) { this.snpName = snpName; this.grps = grps; @@ -50,38 +59,34 @@ public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection groups() { return grps; } + /** + * @return Snapshot name. + */ public String snapshotName() { return snpName; } + /** + * @return List of baseline node IDs that must be alive to complete the operation. + */ public Set requiredNodes() { return Collections.unmodifiableSet(reqNodes); } - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - return Objects.equals(reqId, ((SnapshotRestorePrepareRequest)o).reqId); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return Objects.hash(reqId); - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(SnapshotRestorePrepareRequest.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java index 80dbbb7b30c87d..18e3b20f6109ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -19,8 +19,6 @@ import java.io.Serializable; import java.util.List; -import java.util.Set; -import org.apache.ignite.internal.processors.cache.StoredCacheData; /** * Snapshot restore prepare operation single node validation response. @@ -47,57 +45,4 @@ public SnapshotRestorePrepareResponse(List groups) { public List groups() { return groups; } - - /** */ - static class CacheGroupSnapshotDetails implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Cache group name. */ - private String grpName; - - /** Local partition IDs. */ - private Set parts; - - /** Group cache configurations. */ - private List cfgs; - - private boolean shared; - - /** - * @param cfgs Group cache configurations. - * @param parts Local partition IDs. - */ - public CacheGroupSnapshotDetails(String grpName, boolean shared, List cfgs, Set parts) { - this.grpName = grpName; - this.shared = shared; - this.cfgs = cfgs; - this.parts = parts; - } - - /** - * @return Group cache configurations. - */ - public List configs() { - return cfgs; - } - - /** - * @return Local partition IDs. - */ - public Set parts() { - return parts; - } - - /** - * @return Cache group name. - */ - public String groupName() { - return grpName; - } - - public boolean shared() { - return shared; - } - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java index 9ce4e2d063ff24..69342a8bba8102 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java @@ -19,25 +19,46 @@ import java.io.Serializable; import java.util.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; +/** + * Request to rollback snapshot restore. + */ public class SnapshotRestoreRollbackRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** The reason to rollback operation. */ private final Throwable reason; + /** Request ID. */ private final UUID reqId; + /** + * @param reqId Request ID. + * @param reason The reason to rollback operation. + */ public SnapshotRestoreRollbackRequest(UUID reqId, Throwable reason) { this.reqId = reqId; this.reason = reason; } + /** + * @return Request ID. + */ public UUID requestId() { return reqId; } + /** + * @return The reason to rollback operation. + */ public Throwable reason() { return reason; } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestoreRollbackRequest.class, this); + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java index e39fa302f403e5..a774d520a20386 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java @@ -1072,12 +1072,6 @@ public IgniteInternalFuture changeGlobalState( ); } -// if (state == INACTIVE && ctx.cache().context().snapshotMgr().isSnapshotRestoring()) { -// return new GridFinishedFuture<>( -// new IllegalStateException("The cluster cannot be deactivated until the snapshot restore operation is complete.") -// ); -// } - BaselineTopology blt = (compatibilityMode && !forceChangeBaselineTopology) ? null : calculateNewBaselineTopology(state, baselineNodes, forceChangeBaselineTopology); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index d82b48e96c6b6a..919b3a7eb88d97 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -447,10 +447,19 @@ public enum DistributedProcessType { */ PERFORMANCE_STATISTICS_ROTATE, - START_SNAPSHOT_RESTORE, + /** + * Cache group restore from snapshot prepare process. + */ + RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - END_SNAPSHOT_RESTORE, + /** + * Cache group restore from snapshot perform process. + */ + RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, - UNDO_SNAPSHOT_RESTORE + /** + * Cache group restore from snapshot rollback process. + */ + RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index b47932c6459be8..133e2d8828cb31 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -28,6 +28,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteSnapshot; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryType; @@ -53,8 +54,8 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.END_SNAPSHOT_RESTORE; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.START_SNAPSHOT_RESTORE; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; import static org.apache.ignite.testframework.GridTestUtils.runAsync; /** @@ -93,12 +94,6 @@ private QueryEntity queryEntity(String typeName) { .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); } -// /** @throws Exception If fails. */ -// @Before -// @Override public void beforeTestSnapshot() throws Exception { -// super.beforeTestSnapshot(); -// } - /** {@inheritDoc} */ @Override public void afterTestSnapshot() throws Exception { stopAllGrids(); @@ -111,8 +106,8 @@ public void testBasicClusterSnapshotRestore() throws Exception { IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); - ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + ignite.snapshot().restoreCacheGroups( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); @@ -137,8 +132,8 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { forceCheckpoint(); - ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + ignite.snapshot().restoreCacheGroups( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -156,8 +151,8 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception ignite.cluster().state(ClusterState.INACTIVE); - IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut = + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteException.class, "The cluster should be active"); @@ -176,26 +171,27 @@ public void testRestoreWithMissedPartitions() throws Exception { stopGrid(1); - IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut = + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteCheckedException.class, "not all partitions available"); startGrid(1); - IgniteFuture fut1 = ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut1 = + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, "Cache \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); + log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, + "Cache \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); ignite.cache(dfltCacheCfg.getName()).destroy(); awaitPartitionMapExchange(); - ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + ignite.snapshot().restoreCacheGroups( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } @@ -231,8 +227,8 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { forceCheckpoint(); - ignite.snapshot(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + ignite.snapshot().restoreCacheGroups( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); IgniteCache cache = grid(nodesCnt - 1).cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -271,11 +267,11 @@ public void testRestoreSharedCacheGroup() throws Exception { awaitPartitionMapExchange(); - IgniteSnapshotManager snapshotMgr = ignite.context().cache().context().snapshotMgr(); + IgniteSnapshot snp = ignite.snapshot(); GridTestUtils.assertThrowsAnyCause( log, - () -> snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(MAX_AWAIT_MILLIS), + () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(MAX_AWAIT_MILLIS), IllegalArgumentException.class, "Cache group(s) \"" + cacheName1 + ", " + cacheName2 + "\" not found in snapshot \"" + SNAPSHOT_NAME + "\"" ); @@ -284,7 +280,7 @@ public void testRestoreSharedCacheGroup() throws Exception { awaitPartitionMapExchange(); - snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(MAX_AWAIT_MILLIS); + snp.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(MAX_AWAIT_MILLIS); checkCacheKeys(ignite.cache(cacheName1), CACHE_KEYS_RANGE); checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); @@ -313,8 +309,8 @@ public void testIncompatibleMetasUpdate() throws Exception { return objs[n]; }); - IgniteFuture fut = ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut = + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); fut.get(MAX_AWAIT_MILLIS); @@ -332,7 +328,7 @@ public void testIncompatibleMetasUpdate() throws Exception { ignite.context().cacheObjects().removeType(typeId); - // Create cache with incompatible binary type + // Create cache with incompatible binary type. cache1 = createCacheWithBinaryType(ignite, "cache1", n -> { BinaryObjectBuilder builder = ignite.binary().builder(BIN_TYPE_NAME); @@ -343,14 +339,14 @@ public void testIncompatibleMetasUpdate() throws Exception { return objs[n]; }); - final IgniteFuture fut0 = ignite.context().cache().context().snapshotMgr(). - restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut0 = + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( log, () -> fut0.get(MAX_AWAIT_MILLIS), IgniteException.class, - "Snapshot restore operation was rejected. Incompatible binary types found" + "Cache group restore operation was rejected. Incompatible binary types found" ); ensureCacheDirEmpty(2, dfltCacheCfg.getName()); @@ -388,7 +384,7 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? START_SNAPSHOT_RESTORE : END_SNAPSHOT_RESTORE, grpName); + IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE : RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, grpName); String msgFormat = "Cache start failed. A cache named \"%s\" is currently being restored from a snapshot."; @@ -450,7 +446,7 @@ private void checkBaselineChange(boolean stopNode) throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); - IgniteFuture fut = waitForBlockOnRestore(spi, END_SNAPSHOT_RESTORE, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, dfltCacheCfg.getName()); if (stopNode) { runAsync(() -> stopGrid(3, true)); @@ -459,7 +455,7 @@ private void checkBaselineChange(boolean stopNode) throws Exception { log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteException.class, - "Snapshot restore operation was rejected. Baseline node has left the cluster" + "Cache group restore operation was rejected. Baseline node has left the cluster" ); ensureCacheDirEmpty(3, dfltCacheCfg.getName()); @@ -487,12 +483,12 @@ private void checkBaselineChange(boolean stopNode) throws Exception { @Test public void testClusterStateChangeActiveReadonlyDuringPrepare() throws Exception { - checkReadOnlyDuringRestoring(START_SNAPSHOT_RESTORE); + checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); } @Test public void testClusterStateChangeActiveReadonlyDuringPerform() throws Exception { - checkReadOnlyDuringRestoring(END_SNAPSHOT_RESTORE); + checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); } private void checkReadOnlyDuringRestoring(DistributedProcessType procType) throws Exception { @@ -502,12 +498,12 @@ private void checkReadOnlyDuringRestoring(DistributedProcessType procType) throw @Test public void testClusterDeactivateOnPrepare() throws Exception { - checkDeactivationDuringRestoring(START_SNAPSHOT_RESTORE); + checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); } @Test public void testClusterDeactivateOnPerform() throws Exception { - checkDeactivationDuringRestoring(END_SNAPSHOT_RESTORE); + checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); } private void checkDeactivationDuringRestoring(DistributedProcessType procType) throws Exception { @@ -602,8 +598,6 @@ public void testActivateFromClientWhenRestoring() throws Exception { client.cluster().state(ClusterState.INACTIVE); - IgniteSnapshotManager snapshotMgr = grid(1).context().cache().context().snapshotMgr(); - // todo block distribprocess and try to activate cluster TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); @@ -616,7 +610,8 @@ public void testActivateFromClientWhenRestoring() throws Exception { return false; }); - IgniteFuture fut = snapshotMgr.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + IgniteFuture fut = + grid(1).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); spi.waitForBlocked(); From 0e380e0d92b848028b9c0874f606687e85d0da2f Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 22 Jan 2021 17:35:25 +0300 Subject: [PATCH 10/98] IGNITE-13805 Test with nodeFilter. --- .../snapshot/CacheGroupSnapshotDetails.java | 1 + .../snapshot/IgniteSnapshotManager.java | 22 ++++---- .../SnapshotRestoreCacheGroupProcess.java | 7 ++- .../IgniteClusterSnapshoRestoreSelfTest.java | 52 +++++++++++++++++++ 4 files changed, 70 insertions(+), 12 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java index 2259d177f7443d..57a2b1f13ffc2c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java @@ -37,6 +37,7 @@ class CacheGroupSnapshotDetails implements Serializable { private List cfgs; /** + * @param grpName Cache group name. * @param cfgs Group cache configurations. * @param parts Local partition IDs. */ 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 399ab711fa5fad..df67fca0e87b2d 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 @@ -1130,13 +1130,12 @@ protected void mergeSnapshotMetadata( boolean checkCompatibility, boolean failIfAbsent ) throws IgniteCheckedException { - File snpMetaDir = new File(snapshotLocalDir(snpName), - DFLT_BINARY_METADATA_PATH + File.separator + pdsSettings.folderName()); + File binDir = binaryWorkDir(snapshotLocalDir(snpName).getAbsolutePath(), pdsSettings.folderName()); - if (!snpMetaDir.exists()) { + if (!binDir.exists()) { if (failIfAbsent) { throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + - "directory doesn't exists [snpName=" + snpName + ", dir=" + snpMetaDir + ']'); + "directory doesn't exists [snpName=" + snpName + ", dir=" + binDir + ']'); } return; @@ -1146,7 +1145,7 @@ protected void mergeSnapshotMetadata( ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); - for (File file : snpMetaDir.listFiles()) { + for (File file : binDir.listFiles()) { try (FileInputStream in = new FileInputStream(file)) { BinaryMetadata newMeta = U.unmarshal(marshaller, in, clsLdr); @@ -1176,12 +1175,13 @@ protected void mergeSnapshotMetadata( * @throws IgniteCheckedException If failed. */ protected void restoreCacheGroupFiles(String snpName, String grpName, List newFiles) throws IgniteCheckedException { - File snapshotCacheDir = resolveCacheDir(snapshotLocalDir(snpName), grpName); + File snapshotCacheDir = resolveSnapshotCacheDir(snpName, grpName); if (!snapshotCacheDir.exists()) return; - File cacheDir = resolveCacheDir(new File(cctx.kernalContext().config().getWorkDirectory()), grpName); + File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), DFLT_STORE_DIR + + File.separator + pdsSettings.folderName() + File.separator + snapshotCacheDir.getName(), false); if (!cacheDir.exists()) { cacheDir.mkdir(); @@ -1239,11 +1239,13 @@ protected void rollbackRestoreOperation(Collection files) { } /** - * @param workDir Work directory. + * @param snpName Snapshot name. * @param cacheName Cache (group) name. * @return Local path to the cache directory. */ - private File resolveCacheDir(File workDir, String cacheName) { + private File resolveSnapshotCacheDir(String snpName, String cacheName) { + File workDir = snapshotLocalDir(snpName); + String dbPath = DFLT_STORE_DIR + File.separator + pdsSettings.folderName() + File.separator; File cacheDir = new File(workDir, dbPath + CACHE_DIR_PREFIX + cacheName); @@ -1265,7 +1267,7 @@ private File resolveCacheDir(File workDir, String cacheName) { String grpName ) throws IgniteCheckedException { IgniteConfiguration nodeCfg = cctx.kernalContext().config(); - File cacheDir = resolveCacheDir(snapshotLocalDir(snpName), grpName); + File cacheDir = resolveSnapshotCacheDir(snpName, grpName); if (!cacheDir.exists()) return null; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 4d5181676e04b8..3e62b7a5548939 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -291,14 +291,14 @@ private IgniteInternalFuture prepare(SnapshotRes if (ctx.clientNode()) return new GridFinishedFuture<>(); - stopped = false; - if (inProgress(null)) return errResponse(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); if (!ctx.state().clusterState().state().active()) return errResponse(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); + stopped = false; + // Skip creating future on initiator. if (fut.isDone()) fut = new RestoreSnapshotFuture(); @@ -339,6 +339,9 @@ private IgniteInternalFuture prepare(SnapshotRes private @Nullable SnapshotRestorePrepareResponse prepare0( SnapshotRestorePrepareRequest req ) throws IgniteCheckedException { + if (log.isInfoEnabled()) + log.info("Preparing to restore cache groups [groups=" + F.concat(req.groups(), ", ") + ']'); + List grpCfgs = new ArrayList<>(); IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 133e2d8828cb31..158d33b2479ed9 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.annotations.QuerySqlField; @@ -286,6 +287,57 @@ public void testRestoreSharedCacheGroup() throws Exception { checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); } + /** @throws Exception If fails. */ + @Test + public void testRestoreCacheGroupWithNodeFilter() throws Exception { + String cacheName1 = "cache1"; + String cacheName2 = "cache2"; + + CacheConfiguration cacheCfg1 = txCacheConfig(new CacheConfiguration(cacheName1)).setCacheMode(CacheMode.REPLICATED); + CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration(cacheName2)).setCacheMode(CacheMode.REPLICATED); + + IgniteEx ignite0 = startGrid(0); + IgniteEx ignite1 = startGrid(1); + + ignite0.cluster().state(ClusterState.ACTIVE); + + UUID nodeId0 = ignite0.localNode().id(); + UUID nodeId1 = ignite1.localNode().id(); + + cacheCfg1.setNodeFilter(node -> node.id().equals(nodeId0)); + cacheCfg2.setNodeFilter(node -> node.id().equals(nodeId1)); + + IgniteCache cache1 = ignite0.createCache(cacheCfg1); + putKeys(cache1, 0, CACHE_KEYS_RANGE); + + IgniteCache cache2 = ignite0.createCache(cacheCfg2); + putKeys(cache2, 0, CACHE_KEYS_RANGE); + + ignite0.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + + cache1.destroy(); + cache2.destroy(); + + awaitPartitionMapExchange(); + + forceCheckpoint(); + + // After destroying the cache with a node filter, the configuration file remains on the filtered node. + // todo https://issues.apache.org/jira/browse/IGNITE-14044 + for (String cacheName : new String[] {cacheName1, cacheName2}) { + for (int nodeIdx = 0; nodeIdx < 2; nodeIdx++) + U.delete(resolveCacheDir(grid(nodeIdx), cacheName)); + } + + ignite0.cluster().state(ClusterState.ACTIVE); + + ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(MAX_AWAIT_MILLIS); + ignite1.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(MAX_AWAIT_MILLIS); + + checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); + checkCacheKeys(ignite0.cache(cacheName2), CACHE_KEYS_RANGE); + } + /** @throws Exception If fails. */ @Test public void testIncompatibleMetasUpdate() throws Exception { From ccd774734041d89c876cd30637238ff88a110b3b Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 25 Jan 2021 18:48:15 +0300 Subject: [PATCH 11/98] IGNITE-13805 Decomposition/cleanup/refactoring (wip). --- .../snapshot/IgniteSnapshotManager.java | 8 +- .../SnapshotRestoreCacheGroupProcess.java | 283 +++++++++--------- .../SnapshotRestorePerformRequest.java | 23 +- .../SnapshotRestorePrepareRequest.java | 6 +- .../SnapshotRestoreRollbackRequest.java | 6 +- .../IgniteClusterSnapshoRestoreSelfTest.java | 58 ++-- 6 files changed, 201 insertions(+), 183 deletions(-) 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 df67fca0e87b2d..83c0eedc396c1c 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 @@ -56,6 +56,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -1123,12 +1124,14 @@ public List readSnapshotMetadatas(String snpName) { * @param snpName Snapshot name. * @param checkCompatibility Don't update metadata, just check the compatibility of the snapshot metadata. * @param failIfAbsent Throw an exception if the snapshot metadata folder doesn't exists. + * @param interruptClosure A closure to quickly interrupt the merge process. * @throws IgniteCheckedException If failed. */ protected void mergeSnapshotMetadata( String snpName, boolean checkCompatibility, - boolean failIfAbsent + boolean failIfAbsent, + Supplier interruptClosure ) throws IgniteCheckedException { File binDir = binaryWorkDir(snapshotLocalDir(snpName).getAbsolutePath(), pdsSettings.folderName()); @@ -1146,6 +1149,9 @@ protected void mergeSnapshotMetadata( CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); for (File file : binDir.listFiles()) { + if (interruptClosure.get()) + return; + try (FileInputStream in = new FileInputStream(file)) { BinaryMetadata newMeta = U.unmarshal(marshaller, in, clsLdr); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 3e62b7a5548939..652a396b40e192 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -85,9 +86,6 @@ public class SnapshotRestoreCacheGroupProcess { /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); - /** Stopped flag. */ - private volatile boolean stopped; - /** * @param ctx Kernal context. */ @@ -162,10 +160,11 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames public boolean inProgress(@Nullable String cacheName) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.request() == null) - return false; + return !staleFuture(fut0) && (cacheName == null || fut0.context().containsCache(cacheName)); + } - return cacheName == null || fut0.containsCache(cacheName); + public boolean staleFuture(RestoreSnapshotFuture fut) { + return fut.isDone() || fut.context() == null; } /** @@ -176,12 +175,12 @@ public boolean inProgress(@Nullable String cacheName) { public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullable Throwable err) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.request() == null) + if (staleFuture(fut0)) return; String grpName0 = grpName != null ? grpName : cacheName; - PendingStartCacheGroup pendingGrp = fut0.pendingStartCaches.get(grpName0); + PendingStartCacheGroup pendingGrp = fut0.context().pendingStartCaches.get(grpName0); // If any of shared caches has been started - we cannot rollback changes. if (pendingGrp.caches.remove(cacheName) && err == null) @@ -190,37 +189,35 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab if (!pendingGrp.caches.isEmpty()) return; - if (pendingGrp.canRollback && err != null && fut.rollbackContext() != null) { + if (pendingGrp.canRollback && err != null && fut.context().rollbackContext() != null) { ctx.getSystemExecutorService().submit(() -> { - rollbackChanges(fut0, grpName0); - fut0.onDone(err); }); return; } - fut0.pendingStartCaches.remove(grpName0); + fut0.context().pendingStartCaches.remove(grpName0); - if (fut0.pendingStartCaches.isEmpty()) + if (fut0.context().pendingStartCaches.isEmpty()) fut0.onDone(); } /** * Rollback changes made by process. * - * @param fut Restore future. + * @param opCtx Restore operation context. * @param grpName Cache group name. * @return {@code True} if changes were rolled back, {@code False} if changes have been already rolled back. */ - public boolean rollbackChanges(RestoreSnapshotFuture fut, String grpName) { + public boolean rollbackChanges(OperationContext opCtx, String grpName) { rollbackLock.lock(); try { - if (fut.isDone()) - return false; +// if (staleFuture(fut)) +// return false; - List createdFiles = fut.rollbackContext().remove(grpName); + List createdFiles = opCtx.rollbackContext().remove(grpName); if (F.isEmpty(createdFiles)) return false; @@ -241,13 +238,11 @@ public boolean rollbackChanges(RestoreSnapshotFuture fut, String grpName) { public void onNodeLeft(UUID leftNodeId) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone()) + if (staleFuture(fut0)) return; - SnapshotRestorePrepareRequest req = fut0.request(); - - if (req != null && req.requiredNodes().contains(leftNodeId)) { - fut.handleError(new IgniteException(OP_REJECT_MSG + + if (fut0.context().nodes().contains(leftNodeId)) { + fut.onDone(new IgniteException(OP_REJECT_MSG + "Baseline node has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -258,26 +253,14 @@ public void onNodeLeft(UUID leftNodeId) { * @param reason Interruption reason. */ public void stop(String reason) { - stopped = true; - if (ctx.clientNode()) return; RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone()) - return; - - SnapshotRestorePrepareRequest req = fut0.request(); - - if (req == null) + if (staleFuture(fut0)) return; - log.warning("Snapshot restore process has been interrupted [grps=" + req.groups() + ']'); - - for (String grpName : fut0.request().groups()) - rollbackChanges(fut0, grpName); - fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + reason)); } @@ -297,13 +280,11 @@ private IgniteInternalFuture prepare(SnapshotRes if (!ctx.state().clusterState().state().active()) return errResponse(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); - stopped = false; - // Skip creating future on initiator. if (fut.isDone()) fut = new RestoreSnapshotFuture(); - fut.request(req); + fut.init(req); if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) return new GridFinishedFuture<>(); @@ -357,7 +338,9 @@ private IgniteInternalFuture prepare(SnapshotRes if (grpCfgs.isEmpty()) return null; - ctx.cache().context().snapshotMgr().mergeSnapshotMetadata(req.snapshotName(), true, false); + RestoreSnapshotFuture fut0 = fut; + + ctx.cache().context().snapshotMgr().mergeSnapshotMetadata(req.snapshotName(), true, false, fut0::interrupted); return new SnapshotRestorePrepareResponse(grpCfgs); } @@ -372,14 +355,11 @@ private IgniteInternalFuture prepare(SnapshotRes private void finishPrepare(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (!errs.isEmpty()) { - completeFuture(reqId, errs, fut0); - + if (fut0.interrupted()) return; - } - if (fut0.failure() != null) { - fut0.onDone(fut0.failure()); + if (!errs.isEmpty()) { + completeFuture(reqId, errs, fut0); return; } @@ -396,7 +376,9 @@ private void finishPrepare(UUID reqId, Map } } - List notFoundGroups = new ArrayList<>(fut0.request().groups()); + OperationContext opCtx = fut0.context(); + + Set notFoundGroups = new HashSet<>(opCtx.groups()); try { Collection grpsDetails = mergeNodeResults(res); @@ -419,13 +401,13 @@ private void finishPrepare(UUID reqId, Map notFoundGroups.remove(grpDetails.groupName()); - PendingStartCacheGroup pendingGrp = fut0.pendingStartCaches.get(grpDetails.groupName()); + PendingStartCacheGroup pendingGrp = opCtx.pendingStartCaches.get(grpDetails.groupName()); for (StoredCacheData cacheData : grpDetails.configs()) { String cacheName = cacheData.config().getName(); if (!F.isEmpty(cacheData.config().getGroupName())) { - fut0.addCacheId(CU.cacheId(cacheName)); + opCtx.addCacheId(CU.cacheId(cacheName)); pendingGrp.caches.add(cacheName); } @@ -441,15 +423,15 @@ private void finishPrepare(UUID reqId, Map } if (!notFoundGroups.isEmpty()) { - throw new IllegalArgumentException("Cache group(s) \"" + F.concat(notFoundGroups, ", ") + - "\" not found in snapshot \"" + fut0.request().snapshotName() + "\""); + throw new IllegalArgumentException("Cache group(s) not found in snapshot [groups=" + + F.concat(notFoundGroups, ", ") + ", snapshot=" + opCtx.snapshotName() + ']'); } Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); + Set reqNodes = new HashSet<>(opCtx.nodes()); reqNodes.removeAll(srvNodeIds); @@ -458,7 +440,7 @@ private void finishPrepare(UUID reqId, Map "the cluster [nodeIds=" + F.concat(reqNodes, ", ") + ']'); } - fut0.startConfigs(cacheCfgs); + opCtx.startConfigs(cacheCfgs); } catch (Exception e) { fut0.onDone(e); @@ -466,17 +448,8 @@ private void finishPrepare(UUID reqId, Map return; } - SnapshotRestorePrepareRequest req = fut0.request(); - if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) - performRestoreProc.start(reqId, - new SnapshotRestorePerformRequest( - req.requestId(), - req.snapshotName(), - req.groups(), - req.requiredNodes(), - updateMetadataNode) - ); + performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, updateMetadataNode)); } /** @@ -527,24 +500,22 @@ private Collection mergeNodeResults(Map perform(SnapshotRestorePerformRequest req) { - if (ctx.clientNode() || !req.requiredNodes().contains(ctx.localNodeId())) + if (ctx.clientNode()) return new GridFinishedFuture<>(); - RestoreSnapshotFuture fut0 = fut; - - SnapshotRestorePrepareRequest req0 = fut0.request(); + OperationContext opCtx = fut.context(); - if (req0 == null || !req.requestId().equals(req0.requestId())) + if (!req.requestId().equals(opCtx.requestId())) return errResponse("Unknown snapshot restore operation was rejected."); GridFutureAdapter retFut = new GridFutureAdapter<>(); - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) return new GridFinishedFuture<>(); ctx.getSystemExecutorService().submit(() -> { try { - restore0(req, fut0.rollbackContext()); + restore0(opCtx.snapshotName(), opCtx.groups(), req.updateMetaNodeId(), opCtx.rollbackContext()); retFut.onDone(); } catch (Throwable t) { @@ -556,29 +527,37 @@ private IgniteInternalFuture perform(SnapshotRes } /** - * @param req Request to perform snapshot restore. + * @param snpName Snapshot name. + * @param groups List of cache group names to restore from the snapshot. + * @param updateMetaNodeId Node ID from which to update the binary metadata. * @param opCtx Restore operation context. * @throws IgniteCheckedException If failed. */ - private void restore0(SnapshotRestorePerformRequest req, - RestoreOperationContext opCtx) throws IgniteCheckedException { + private void restore0( + String snpName, + Set groups, + UUID updateMetaNodeId, + RestoreOperationContext opCtx + ) throws IgniteCheckedException { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - if (ctx.localNodeId().equals(req.updateMetaNodeId()) && !stopped) - snapshotMgr.mergeSnapshotMetadata(req.snapshotName(), false, true); + RestoreSnapshotFuture fut0 = fut; + + if (ctx.localNodeId().equals(updateMetaNodeId) && !fut0.interrupted()) + snapshotMgr.mergeSnapshotMetadata(snpName, false, true, fut0::interrupted); - for (String grpName : req.groups()) { + for (String grpName : groups) { rollbackLock.lock(); try { - if (stopped) + if (fut0.interrupted()) return; List newFiles = new ArrayList<>(); opCtx.put(grpName, newFiles); - snapshotMgr.restoreCacheGroupFiles(req.snapshotName(), grpName, newFiles); + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles); } finally { rollbackLock.unlock(); @@ -594,35 +573,25 @@ private void restore0(SnapshotRestorePerformRequest req, private void finishPerform(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - Throwable failure = F.first(errs.values()); - - if (failure == null) - failure = fut0.failure(); - - if (failure == null && !res.keySet().containsAll(fut0.request().requiredNodes())) { - Set reqNodes = new HashSet<>(fut0.request().requiredNodes()); - - reqNodes.removeAll(res.keySet()); - - log.warning("Node(s) left the cluster, " + - "snapshot restore operation should be reverted [nodeIds=" + F.concat(reqNodes, ", ")); + if (fut0.isDone() || fut0.interrupted()) + return; - fut0.handleError(failure = new IgniteException(new IgniteException(OP_REJECT_MSG + - "Baseline node has left the cluster [nodeId(s)=" + F.concat(reqNodes, ", ") + ']'))); - } + Exception failure = F.first(errs.values()); if (failure != null) { if (U.isLocalNodeCoordinator(ctx.discovery())) { log.info("Starting snapshot restore rollback routine."); - rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(fut0.request().requestId(), failure)); + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(fut0.id(), failure)); } return; } - if (U.isLocalNodeCoordinator(ctx.discovery())) - ctx.cache().dynamicStartCachesByStoredConf(fut0.startConfigs(), true, true, false, null, true); + if (!U.isLocalNodeCoordinator(ctx.discovery())) + return; + + ctx.cache().dynamicStartCachesByStoredConf(fut0.context().startConfigs(), true, true, false, null, true); } /** @@ -635,17 +604,18 @@ private IgniteInternalFuture rollback(SnapshotR RestoreSnapshotFuture fut0 = fut; - SnapshotRestorePrepareRequest req0 = fut0.request(); + if (fut0.isDone() || fut0.interrupted()) + return new GridFinishedFuture<>(); + + OperationContext opCtx = fut0.context(); - if (req0 == null || !req.requestId().equals(req0.requestId())) + if (!req.requestId().equals(opCtx.requestId())) return errResponse("Unknown snapshot restore rollback operation was rejected."); - for (String grpName : req0.groups()) - rollbackChanges(fut0, grpName); - - fut0.handleError(req.reason()); + for (String grpName : opCtx.groups()) + rollbackChanges(opCtx, grpName); - return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse()); + return new GridFinishedFuture<>(req.reason()); } /** @@ -656,13 +626,8 @@ private IgniteInternalFuture rollback(SnapshotR private void finishRollback(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (!F.isEmpty(errs)) { + if (!F.isEmpty(errs)) completeFuture(reqId, errs, fut0); - - return; - } - - fut0.onDone(fut0.failure()); } /** @@ -724,27 +689,32 @@ public void put(String grpName, List files) { } } - /** */ - private static class RestoreSnapshotFuture extends GridFutureAdapter { + private static class OperationContext { private final RestoreOperationContext rollbackCtx = new RestoreOperationContext(); - private volatile Throwable err; + private final Map pendingStartCaches = new ConcurrentHashMap<>(); - private volatile SnapshotRestorePrepareRequest req; + private final Set cacheIds = new GridConcurrentHashSet<>(); - public Throwable failure() { - return err; - } + private final String snpName; + + private final UUID requestId; + + private final Set reqNodes; private volatile Collection cacheCfgsToStart; - private final Map pendingStartCaches = new ConcurrentHashMap<>(); + public OperationContext(UUID requestId, String snpName, Set reqNodes, Collection grps) { + for (String grpName : grps) { + cacheIds.add(CU.cacheId(grpName)); - public SnapshotRestorePrepareRequest request() { - return req; - } + pendingStartCaches.put(grpName, new PendingStartCacheGroup()); + } - public Set cacheIds = new GridConcurrentHashSet<>(); + this.requestId = requestId; + this.reqNodes = reqNodes; + this.snpName = snpName; + } public boolean containsCache(String name) { return cacheIds.contains(CU.cacheId(name)); @@ -754,39 +724,76 @@ public void addCacheId(int cacheId) { cacheIds.add(cacheId); } - public void request(SnapshotRestorePrepareRequest req) { - this.req = req; + public void startConfigs(Collection ccfgs) { + cacheCfgsToStart = ccfgs; + } + + public Collection startConfigs() { + return cacheCfgsToStart; + } - for (String grpName : req.groups()) { - cacheIds.add(CU.cacheId(grpName)); + public RestoreOperationContext rollbackContext() { + return rollbackCtx; + } - pendingStartCaches.put(grpName, new PendingStartCacheGroup()); - } + public Set nodes() { + return reqNodes; + } + + public String snapshotName() { + return snpName; } /** @return Request ID. */ - public UUID id() { - return req != null ? req.requestId() : null; + public UUID requestId() { + return requestId; } - public void handleError(Throwable err) { - this.err = err; + public Set groups() { + return pendingStartCaches.keySet(); } + } - public void startConfigs(Collection ccfgs) { - cacheCfgsToStart = ccfgs; + /** */ + private class RestoreSnapshotFuture extends GridFutureAdapter { + private volatile OperationContext ctx; + + private final AtomicReference errRef = new AtomicReference<>(); + + public UUID id() { + return ctx != null ? ctx.requestId() : null; } - public Collection startConfigs() { - return cacheCfgsToStart; + public OperationContext context() { + return ctx; } - public RestoreOperationContext rollbackContext() { - return rollbackCtx; + public void init(SnapshotRestorePrepareRequest req) { + ctx = new OperationContext(req.requestId(), req.snapshotName(), req.requiredNodes(), req.groups()); + } + + public boolean interrupted() { + return errRef.get() != null; } @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { - return super.onDone(res, err, cancel); + if (err == null) + return super.onDone(res, err, cancel); + + if (errRef.compareAndSet(null, err)) { + OperationContext opCtx0 = ctx; + + Set grpNames = opCtx0.groups(); + + log.warning("Snapshot restore process has been interrupted [grps=" + grpNames + ']'); + + for (String grpName : grpNames) + rollbackChanges(opCtx0, grpName); + + return super.onDone(res, err, cancel); + } + + return false; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java index c42e17f680b34d..a3df9bc9b46246 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java @@ -17,34 +17,39 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.util.Collection; -import java.util.Set; +import java.io.Serializable; import java.util.UUID; import org.apache.ignite.internal.util.typedef.internal.S; /** * Request to perform snapshot restore. */ -public class SnapshotRestorePerformRequest extends SnapshotRestorePrepareRequest { +public class SnapshotRestorePerformRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** Request ID. */ + private final UUID reqId; + /** Node ID from which to update the binary metadata. */ private final UUID updateMetaNodeId; /** * @param reqId Request ID. - * @param snpName Snapshot name. - * @param grps List of cache group names to restore from the snapshot. - * @param reqNodes List of baseline node IDs that must be alive to complete the operation. * @param updateMetaNodeId Node ID from which to update the binary metadata. */ - public SnapshotRestorePerformRequest(UUID reqId, String snpName, Collection grps, Set reqNodes, UUID updateMetaNodeId) { - super(reqId, snpName, grps, reqNodes); - + public SnapshotRestorePerformRequest(UUID reqId, UUID updateMetaNodeId) { + this.reqId = reqId; this.updateMetaNodeId = updateMetaNodeId; } + /** + * @return Request ID. + */ + public UUID requestId() { + return reqId; + } + /** * @return Node ID from which to update the binary metadata. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index 57a68aabb83fca..62a86ff9430e73 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -32,6 +32,9 @@ public class SnapshotRestorePrepareRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; + /** Request ID. */ + private final UUID reqId; + /** Snapshot name. */ private final String snpName; @@ -43,9 +46,6 @@ public class SnapshotRestorePrepareRequest implements Serializable { @GridToStringInclude private final Set reqNodes; - /** Request ID. */ - private final UUID reqId; - /** * @param reqId Request ID. * @param snpName Snapshot name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java index 69342a8bba8102..971714868bb27c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java @@ -28,12 +28,12 @@ public class SnapshotRestoreRollbackRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; - /** The reason to rollback operation. */ - private final Throwable reason; - /** Request ID. */ private final UUID reqId; + /** The reason to rollback operation. */ + private final Throwable reason; + /** * @param reqId Request ID. * @param reason The reason to rollback operation. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 158d33b2479ed9..63fe45ee6e369a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -64,7 +64,7 @@ */ public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTest { /** Timeout. */ - private static final long MAX_AWAIT_MILLIS = 15_000; + private static final long TIMEOUT = 15_000; private static final String BIN_TYPE_NAME = "customType"; @@ -108,7 +108,7 @@ public void testBasicClusterSnapshotRestore() throws Exception { IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); @@ -134,7 +134,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { forceCheckpoint(); ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -148,7 +148,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception { IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); ignite.cluster().state(ClusterState.INACTIVE); @@ -156,7 +156,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteException.class, "The cluster should be active"); + log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); } /** @throws Exception If fails. */ @@ -164,7 +164,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception public void testRestoreWithMissedPartitions() throws Exception { IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg.setBackups(0)); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); putKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); @@ -176,7 +176,7 @@ public void testRestoreWithMissedPartitions() throws Exception { ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut.get(MAX_AWAIT_MILLIS), IgniteCheckedException.class, "not all partitions available"); + log, () -> fut.get(TIMEOUT), IgniteCheckedException.class, "not all partitions available"); startGrid(1); @@ -184,7 +184,7 @@ public void testRestoreWithMissedPartitions() throws Exception { ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut1.get(MAX_AWAIT_MILLIS), IllegalStateException.class, + log, () -> fut1.get(TIMEOUT), IllegalStateException.class, "Cache \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); ignite.cache(dfltCacheCfg.getName()).destroy(); @@ -192,7 +192,7 @@ public void testRestoreWithMissedPartitions() throws Exception { awaitPartitionMapExchange(); ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } @@ -208,7 +208,7 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { IgniteEx ignite = startGridsWithCache(nodesCnt - 2, keysCnt, valueBuilder, dfltCacheCfg); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); startGrid(nodesCnt - 2); startGrid(nodesCnt - 1); @@ -229,7 +229,7 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { forceCheckpoint(); ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = grid(nodesCnt - 1).cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -262,7 +262,7 @@ public void testRestoreSharedCacheGroup() throws Exception { IgniteCache cache2 = ignite.cache(cacheName2); putKeys(cache2, 0, CACHE_KEYS_RANGE); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); cache1.destroy(); @@ -272,16 +272,16 @@ public void testRestoreSharedCacheGroup() throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(MAX_AWAIT_MILLIS), + () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(grpName, cacheName1, cacheName2)).get(TIMEOUT), IllegalArgumentException.class, - "Cache group(s) \"" + cacheName1 + ", " + cacheName2 + "\" not found in snapshot \"" + SNAPSHOT_NAME + "\"" + "Cache group(s) not found in snapshot" ); cache2.destroy(); awaitPartitionMapExchange(); - snp.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(MAX_AWAIT_MILLIS); + snp.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(TIMEOUT); checkCacheKeys(ignite.cache(cacheName1), CACHE_KEYS_RANGE); checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); @@ -313,7 +313,7 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { IgniteCache cache2 = ignite0.createCache(cacheCfg2); putKeys(cache2, 0, CACHE_KEYS_RANGE); - ignite0.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite0.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); cache1.destroy(); cache2.destroy(); @@ -331,8 +331,8 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { ignite0.cluster().state(ClusterState.ACTIVE); - ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(MAX_AWAIT_MILLIS); - ignite1.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(MAX_AWAIT_MILLIS); + ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); + ignite1.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); checkCacheKeys(ignite0.cache(cacheName2), CACHE_KEYS_RANGE); @@ -364,7 +364,7 @@ public void testIncompatibleMetasUpdate() throws Exception { IgniteFuture fut = ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); - fut.get(MAX_AWAIT_MILLIS); + fut.get(TIMEOUT); // Ensure that existing type has been updated BinaryType type = ignite.context().cacheObjects().metadata(typeId); @@ -396,7 +396,7 @@ public void testIncompatibleMetasUpdate() throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> fut0.get(MAX_AWAIT_MILLIS), + () -> fut0.get(TIMEOUT), IgniteException.class, "Cache group restore operation was rejected. Incompatible binary types found" ); @@ -464,7 +464,7 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { if (prepare) { GridTestUtils.assertThrowsAnyCause( log, - () -> fut.get(MAX_AWAIT_MILLIS), + () -> fut.get(TIMEOUT), IgniteException.class, "Cache \"" + cacheName + "\" should be destroyed manually before perform restore operation." ); @@ -473,7 +473,7 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { ensureCacheDirEmpty(1, grpName); } else { - fut.get(MAX_AWAIT_MILLIS); + fut.get(TIMEOUT); checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); } @@ -505,7 +505,7 @@ private void checkBaselineChange(boolean stopNode) throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> fut.get(MAX_AWAIT_MILLIS), + () -> fut.get(TIMEOUT), IgniteException.class, "Cache group restore operation was rejected. Baseline node has left the cluster" ); @@ -524,7 +524,7 @@ private void checkBaselineChange(boolean stopNode) throws Exception { spi.stopBlock(); - fut.get(MAX_AWAIT_MILLIS); + fut.get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); @@ -574,13 +574,13 @@ private void checkClusterStateChange(ClusterState state, DistributedProcessType spi.stopBlock(); - GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(MAX_AWAIT_MILLIS), expCls, expMsg); + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), expCls, expMsg); ensureCacheDirEmpty(2, dfltCacheCfg.getName()); ignite.cluster().state(ClusterState.ACTIVE); - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(MAX_AWAIT_MILLIS); + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } @@ -614,7 +614,7 @@ private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws Igni private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder, dfltCacheCfg); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); ignite.cache(dfltCacheCfg.getName()).destroy(); @@ -644,7 +644,7 @@ public void testActivateFromClientWhenRestoring() throws Exception { IgniteEx client = startClientGrid("client"); - client.snapshot().createSnapshot(SNAPSHOT_NAME).get(MAX_AWAIT_MILLIS); + client.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); putKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); @@ -680,7 +680,7 @@ public void testActivateFromClientWhenRestoring() throws Exception { spi.stopBlock(); - fut.get(MAX_AWAIT_MILLIS); + fut.get(TIMEOUT); client.cluster().state(ClusterState.ACTIVE); From 14dd1196d340b8594a4b6f732fbdcee25100e38c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 25 Jan 2021 18:50:31 +0300 Subject: [PATCH 12/98] IGNITE-13805 No need to distrib rollback. --- .../SnapshotRestoreCacheGroupProcess.java | 48 +------------- .../SnapshotRestoreRollbackRequest.java | 64 ------------------- .../SnapshotRestoreRollbackResponse.java | 28 -------- .../util/distributed/DistributedProcess.java | 7 +- 4 files changed, 2 insertions(+), 145 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 652a396b40e192..d0667845dd57be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -56,7 +56,6 @@ import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; 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_ROLLBACK; /** * Distributed process to restore cache group from the snapshot. @@ -74,9 +73,6 @@ public class SnapshotRestoreCacheGroupProcess { /** Cache group restore perform phase. */ private final DistributedProcess performRestoreProc; - /** Cache group restore rollback phase. */ - private final DistributedProcess rollbackRestoreProc; - /** Logger. */ private final IgniteLogger log; @@ -98,8 +94,6 @@ public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, this::prepare, this::finishPrepare); performRestoreProc = new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, this::perform, this::finishPerform); - rollbackRestoreProc = - new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK, this::rollback, this::finishRollback); fut.onDone(); } @@ -579,11 +573,7 @@ private void finishPerform(UUID reqId, Map Exception failure = F.first(errs.values()); if (failure != null) { - if (U.isLocalNodeCoordinator(ctx.discovery())) { - log.info("Starting snapshot restore rollback routine."); - - rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(fut0.id(), failure)); - } + fut0.onDone(failure); return; } @@ -594,42 +584,6 @@ private void finishPerform(UUID reqId, Map ctx.cache().dynamicStartCachesByStoredConf(fut0.context().startConfigs(), true, true, false, null, true); } - /** - * @param req Request to rollback snapshot restore. - * @return Result future. - */ - private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { - if (ctx.clientNode()) - return new GridFinishedFuture<>(); - - RestoreSnapshotFuture fut0 = fut; - - if (fut0.isDone() || fut0.interrupted()) - return new GridFinishedFuture<>(); - - OperationContext opCtx = fut0.context(); - - if (!req.requestId().equals(opCtx.requestId())) - return errResponse("Unknown snapshot restore rollback operation was rejected."); - - for (String grpName : opCtx.groups()) - rollbackChanges(opCtx, grpName); - - return new GridFinishedFuture<>(req.reason()); - } - - /** - * @param reqId Request ID. - * @param res Results. - * @param errs Errors. - */ - private void finishRollback(UUID reqId, Map res, Map errs) { - RestoreSnapshotFuture fut0 = fut; - - if (!F.isEmpty(errs)) - completeFuture(reqId, errs, fut0); - } - /** * @param reqId Request id. * @param err Exception. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java deleted file mode 100644 index 971714868bb27c..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java +++ /dev/null @@ -1,64 +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.Serializable; -import java.util.UUID; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * Request to rollback snapshot restore. - */ -public class SnapshotRestoreRollbackRequest implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Request ID. */ - private final UUID reqId; - - /** The reason to rollback operation. */ - private final Throwable reason; - - /** - * @param reqId Request ID. - * @param reason The reason to rollback operation. - */ - public SnapshotRestoreRollbackRequest(UUID reqId, Throwable reason) { - this.reqId = reqId; - this.reason = reason; - } - - /** - * @return Request ID. - */ - public UUID requestId() { - return reqId; - } - - /** - * @return The reason to rollback operation. - */ - public Throwable reason() { - return reason; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotRestoreRollbackRequest.class, this); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java deleted file mode 100644 index 755d79e66bcc8c..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java +++ /dev/null @@ -1,28 +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.Serializable; - -/** - * Snapshot restore rollback operation single node response. - */ -public class SnapshotRestoreRollbackResponse implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index 919b3a7eb88d97..ed3eab5a6e061b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -455,11 +455,6 @@ public enum DistributedProcessType { /** * Cache group restore from snapshot perform process. */ - RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, - - /** - * Cache group restore from snapshot rollback process. - */ - RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK + RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM } } From f94398126c4517c4143266cec68341aeb0d468fa Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 25 Jan 2021 19:19:28 +0300 Subject: [PATCH 13/98] IGNITE-13805 Code cleanup. --- .../SnapshotRestoreCacheGroupProcess.java | 263 +++--------------- .../snapshot/SnapshotRestoreContext.java | 234 ++++++++++++++++ .../IgniteClusterSnapshoRestoreSelfTest.java | 52 ---- 3 files changed, 271 insertions(+), 278 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index d0667845dd57be..83ba83efd42343 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.io.File; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -26,9 +25,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; @@ -39,12 +36,12 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; -import org.apache.ignite.internal.util.GridConcurrentHashSet; 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.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.future.IgniteFutureImpl; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -79,9 +76,6 @@ public class SnapshotRestoreCacheGroupProcess { /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(); - /** Restore operation lock. */ - private final ReentrantLock rollbackLock = new ReentrantLock(); - /** * @param ctx Kernal context. */ @@ -172,56 +166,7 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab if (staleFuture(fut0)) return; - String grpName0 = grpName != null ? grpName : cacheName; - - PendingStartCacheGroup pendingGrp = fut0.context().pendingStartCaches.get(grpName0); - - // If any of shared caches has been started - we cannot rollback changes. - if (pendingGrp.caches.remove(cacheName) && err == null) - pendingGrp.canRollback = false; - - if (!pendingGrp.caches.isEmpty()) - return; - - if (pendingGrp.canRollback && err != null && fut.context().rollbackContext() != null) { - ctx.getSystemExecutorService().submit(() -> { - fut0.onDone(err); - }); - - return; - } - - fut0.context().pendingStartCaches.remove(grpName0); - - if (fut0.context().pendingStartCaches.isEmpty()) - fut0.onDone(); - } - - /** - * Rollback changes made by process. - * - * @param opCtx Restore operation context. - * @param grpName Cache group name. - * @return {@code True} if changes were rolled back, {@code False} if changes have been already rolled back. - */ - public boolean rollbackChanges(OperationContext opCtx, String grpName) { - rollbackLock.lock(); - - try { -// if (staleFuture(fut)) -// return false; - - List createdFiles = opCtx.rollbackContext().remove(grpName); - - if (F.isEmpty(createdFiles)) - return false; - - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(createdFiles); - } finally { - rollbackLock.unlock(); - } - - return true; + fut0.context().processCacheStart(cacheName, grpName, err, ctx.getSystemExecutorService(), fut0); } /** @@ -278,9 +223,12 @@ private IgniteInternalFuture prepare(SnapshotRes if (fut.isDone()) fut = new RestoreSnapshotFuture(); - fut.init(req); + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) + fut.init(new SnapshotRestoreContext( + req.requestId(), req.snapshotName(), req.requiredNodes(), req.groups(), snpMgr)); + + if (!snpMgr.snapshotLocalDir(req.snapshotName()).exists()) return new GridFinishedFuture<>(); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -349,11 +297,11 @@ private IgniteInternalFuture prepare(SnapshotRes private void finishPrepare(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (fut0.interrupted()) + if (fut0.interrupted() || !reqId.equals(fut0.context().requestId()) || fut0.isDone()) return; if (!errs.isEmpty()) { - completeFuture(reqId, errs, fut0); + fut0.onDone(F.firstValue(errs)); return; } @@ -370,7 +318,7 @@ private void finishPrepare(UUID reqId, Map } } - OperationContext opCtx = fut0.context(); + SnapshotRestoreContext opCtx = fut0.context(); Set notFoundGroups = new HashSet<>(opCtx.groups()); @@ -395,15 +343,11 @@ private void finishPrepare(UUID reqId, Map notFoundGroups.remove(grpDetails.groupName()); - PendingStartCacheGroup pendingGrp = opCtx.pendingStartCaches.get(grpDetails.groupName()); - for (StoredCacheData cacheData : grpDetails.configs()) { String cacheName = cacheData.config().getName(); - if (!F.isEmpty(cacheData.config().getGroupName())) { - opCtx.addCacheId(CU.cacheId(cacheName)); - pendingGrp.caches.add(cacheName); - } + if (!F.isEmpty(cacheData.config().getGroupName())) + opCtx.addSharedCache(cacheName, grpDetails.groupName()); cacheCfgs.add(cacheData); @@ -497,7 +441,8 @@ private IgniteInternalFuture perform(SnapshotRes if (ctx.clientNode()) return new GridFinishedFuture<>(); - OperationContext opCtx = fut.context(); + RestoreSnapshotFuture fut0 = fut; + SnapshotRestoreContext opCtx = fut0.context(); if (!req.requestId().equals(opCtx.requestId())) return errResponse("Unknown snapshot restore operation was rejected."); @@ -507,9 +452,11 @@ private IgniteInternalFuture perform(SnapshotRes if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) return new GridFinishedFuture<>(); + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + ctx.getSystemExecutorService().submit(() -> { try { - restore0(opCtx.snapshotName(), opCtx.groups(), req.updateMetaNodeId(), opCtx.rollbackContext()); + opCtx.restore(updateMeta, fut0::interrupted); retFut.onDone(); } catch (Throwable t) { @@ -520,45 +467,6 @@ private IgniteInternalFuture perform(SnapshotRes return retFut; } - /** - * @param snpName Snapshot name. - * @param groups List of cache group names to restore from the snapshot. - * @param updateMetaNodeId Node ID from which to update the binary metadata. - * @param opCtx Restore operation context. - * @throws IgniteCheckedException If failed. - */ - private void restore0( - String snpName, - Set groups, - UUID updateMetaNodeId, - RestoreOperationContext opCtx - ) throws IgniteCheckedException { - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - - RestoreSnapshotFuture fut0 = fut; - - if (ctx.localNodeId().equals(updateMetaNodeId) && !fut0.interrupted()) - snapshotMgr.mergeSnapshotMetadata(snpName, false, true, fut0::interrupted); - - for (String grpName : groups) { - rollbackLock.lock(); - - try { - if (fut0.interrupted()) - return; - - List newFiles = new ArrayList<>(); - - opCtx.put(grpName, newFiles); - - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles); - } - finally { - rollbackLock.unlock(); - } - } - } - /** * @param reqId Request ID. * @param res Results. @@ -584,19 +492,6 @@ private void finishPerform(UUID reqId, Map ctx.cache().dynamicStartCachesByStoredConf(fut0.context().startConfigs(), true, true, false, null, true); } - /** - * @param reqId Request id. - * @param err Exception. - * @param fut Key change future. - * @return {@code True} if future was completed by this call. - */ - private boolean completeFuture(UUID reqId, Map err, RestoreSnapshotFuture fut) { - if (!reqId.equals(fut.id()) || fut.isDone()) - return false; - - return !F.isEmpty(err) ? fut.onDone(F.firstValue(err)) : fut.onDone(); - } - /** * @param msg Error message. * @param Type of the future. @@ -615,134 +510,50 @@ private IgniteInternalFuture errResponse(Exception ex) { return new GridFinishedFuture<>(ex); } - /** */ - private static class PendingStartCacheGroup { - volatile boolean canRollback = true; - - Set caches = new GridConcurrentHashSet<>(); - } - - /** */ - private static class RestoreOperationContext { - private final Map> newGrpFiles = new HashMap<>(); - - public List get(String grpName) { - return newGrpFiles.get(grpName); - } - - public List remove(String grpName) { - return newGrpFiles.remove(grpName); - } - - public boolean isEmpty() { - return newGrpFiles.isEmpty(); - } - - public void put(String grpName, List files) { - newGrpFiles.put(grpName, files); - } - } - - private static class OperationContext { - private final RestoreOperationContext rollbackCtx = new RestoreOperationContext(); - - private final Map pendingStartCaches = new ConcurrentHashMap<>(); - - private final Set cacheIds = new GridConcurrentHashSet<>(); - - private final String snpName; - - private final UUID requestId; - - private final Set reqNodes; - - private volatile Collection cacheCfgsToStart; - - public OperationContext(UUID requestId, String snpName, Set reqNodes, Collection grps) { - for (String grpName : grps) { - cacheIds.add(CU.cacheId(grpName)); - - pendingStartCaches.put(grpName, new PendingStartCacheGroup()); - } - - this.requestId = requestId; - this.reqNodes = reqNodes; - this.snpName = snpName; - } - - public boolean containsCache(String name) { - return cacheIds.contains(CU.cacheId(name)); - } - - public void addCacheId(int cacheId) { - cacheIds.add(cacheId); - } - - public void startConfigs(Collection ccfgs) { - cacheCfgsToStart = ccfgs; - } - - public Collection startConfigs() { - return cacheCfgsToStart; - } - - public RestoreOperationContext rollbackContext() { - return rollbackCtx; - } - - public Set nodes() { - return reqNodes; - } - - public String snapshotName() { - return snpName; - } - - /** @return Request ID. */ - public UUID requestId() { - return requestId; - } - - public Set groups() { - return pendingStartCaches.keySet(); - } - } - /** */ private class RestoreSnapshotFuture extends GridFutureAdapter { - private volatile OperationContext ctx; - + /** The exception that led to the interruption of the process. */ private final AtomicReference errRef = new AtomicReference<>(); - public UUID id() { - return ctx != null ? ctx.requestId() : null; - } + /** Snapshot restore operation context. */ + @GridToStringInclude + private volatile SnapshotRestoreContext ctx; - public OperationContext context() { + /** + * @return Snapshot restore operation context. + */ + public SnapshotRestoreContext context() { return ctx; } - public void init(SnapshotRestorePrepareRequest req) { - ctx = new OperationContext(req.requestId(), req.snapshotName(), req.requiredNodes(), req.groups()); + /** + * @param ctx Snapshot restore operation context. + */ + public void init(SnapshotRestoreContext ctx) { + this.ctx = ctx; } + /** + * @return Interrupted flag. + */ public boolean interrupted() { return errRef.get() != null; } + /** {@inheritDoc} */ @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { if (err == null) return super.onDone(res, err, cancel); if (errRef.compareAndSet(null, err)) { - OperationContext opCtx0 = ctx; + SnapshotRestoreContext opCtx0 = ctx; Set grpNames = opCtx0.groups(); - log.warning("Snapshot restore process has been interrupted [grps=" + grpNames + ']'); + log.error("Snapshot restore process has been interrupted [grps=" + grpNames + ']', err); for (String grpName : grpNames) - rollbackChanges(opCtx0, grpName); + opCtx0.rollback(grpName); return super.onDone(res, err, cancel); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java new file mode 100644 index 00000000000000..36c4447b1339a3 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -0,0 +1,234 @@ +/* + * 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.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Supplier; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.util.GridConcurrentHashSet; +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.S; +import org.jetbrains.annotations.Nullable; + +/** + * Cache restore from snapshot operation context. + */ +class SnapshotRestoreContext { + private final Map> newGrpFiles = new HashMap<>(); + + private final Map pendingStartCaches = new ConcurrentHashMap<>(); + + private final Set cacheIds = new GridConcurrentHashSet<>(); + + /** Request ID. */ + private final UUID reqId; + + /** Snapshot name. */ + private final String snpName; + + /** List of baseline node IDs that must be alive to complete the operation. */ + private final Set reqNodes; + + /** Snapshot manager. */ + private final IgniteSnapshotManager snapshotMgr; + + private volatile Collection cacheCfgsToStart; + + /** Restore operation lock. */ + private final ReentrantLock rollbackLock = new ReentrantLock(); + + /** + * @param reqId Request ID. + * @param snpName Snapshot name. + * @param reqNodes List of baseline node IDs that must be alive to complete the operation. + * @param grps List of cache group names to restore from the snapshot. + * @param snapshotMgr Snapshot manager. + */ + public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Collection grps, + IgniteSnapshotManager snapshotMgr) { + for (String grpName : grps) { + cacheIds.add(CU.cacheId(grpName)); + pendingStartCaches.put(grpName, new PendingStartCacheGroup()); + } + + this.reqId = reqId; + this.reqNodes = reqNodes; + this.snpName = snpName; + this.snapshotMgr = snapshotMgr; + } + + /** @return Request ID. */ + public UUID requestId() { + return reqId; + } + + public Set nodes() { + return reqNodes; + } + + public String snapshotName() { + return snpName; + } + + public Set groups() { + return pendingStartCaches.keySet(); + } + + public void addSharedCache(String cacheName, String grpName) { + cacheIds.add(CU.cacheId(cacheName)); + + PendingStartCacheGroup sharedGrp = pendingStartCaches.get(grpName); + + assert sharedGrp != null : grpName; + + sharedGrp.caches.add(cacheName); + } + + public boolean containsCache(String name) { + return cacheIds.contains(CU.cacheId(name)); + } + + public void startConfigs(Collection ccfgs) { + cacheCfgsToStart = ccfgs; + } + + public Collection startConfigs() { + return cacheCfgsToStart; + } + + public void processCacheStart(String cacheName, @Nullable String grpName, @Nullable Throwable err, ExecutorService svc, GridFutureAdapter finishFut) { + String grpName0 = grpName != null ? grpName : cacheName; + + PendingStartCacheGroup pendingGrp = pendingStartCaches.get(grpName0); + + // If any of shared caches has been started - we cannot rollback changes. + if (pendingGrp.caches.remove(cacheName) && err == null) + pendingGrp.canRollback = false; + + if (!pendingGrp.caches.isEmpty()) + return; + + if (err != null && pendingGrp.canRollback) { + svc.submit(() -> { + rollbackLock.lock(); + + try { + pendingStartCaches.remove(grpName0); + + rollback(grpName0); + + if (pendingStartCaches.isEmpty()) + finishFut.onDone(err); + } + finally { + rollbackLock.unlock(); + } + }); + + return; + } + + rollbackLock.lock(); + + try { + pendingStartCaches.remove(grpName0); + + if (pendingStartCaches.isEmpty()) + finishFut.onDone(); + } finally { + rollbackLock.unlock(); + } + } + + /** + * @param updateMetadata Update binary metadata flag. + * @param interruptClosure A closure to quickly interrupt the merge process. + * @throws IgniteCheckedException If failed. + */ + public void restore(boolean updateMetadata, Supplier interruptClosure) throws IgniteCheckedException { + if (interruptClosure.get()) + return; + + if (updateMetadata) + snapshotMgr.mergeSnapshotMetadata(snpName, false, true, interruptClosure); + + for (String grpName : groups()) { + rollbackLock.lock(); + + try { + if (interruptClosure.get()) + return; + + List newFiles = new ArrayList<>(); + + newGrpFiles.put(grpName, newFiles); + + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles); + } + finally { + rollbackLock.unlock(); + } + } + } + + /** + * Rollback changes made by process. + * + * @param grpName Cache group name. + */ + public void rollback(String grpName) { + rollbackLock.lock(); + + try { + List createdFiles = newGrpFiles.remove(grpName); + + if (F.isEmpty(createdFiles)) + return; + + snapshotMgr.rollbackRestoreOperation(createdFiles); + } finally { + rollbackLock.unlock(); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestoreContext.class, this); + } + + /** */ + private static class PendingStartCacheGroup { + volatile boolean canRollback = true; + + Set caches = new GridConcurrentHashSet<>(); + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 63fe45ee6e369a..a90bf3c7963686 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -635,58 +635,6 @@ private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi s return fut; } - /** @throws Exception If fails. */ - @Test - // todo - @Ignore - public void testActivateFromClientWhenRestoring() throws Exception { - IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg); - - IgniteEx client = startClientGrid("client"); - - client.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - - putKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); - - client.cluster().state(ClusterState.INACTIVE); - - // todo block distribprocess and try to activate cluster - TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - - spi.blockMessages((node, msg) -> { - if (msg instanceof SingleNodeMessage) - return true; - - System.out.println(">xxx> " + node.id()); - - return false; - }); - - IgniteFuture fut = - grid(1).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); - - spi.waitForBlocked(); - - GridTestUtils.assertThrowsAnyCause( - log, - () -> { - client.cluster().state(ClusterState.ACTIVE); - - return null; - }, - IllegalStateException.class, - "The cluster cannot be activated until the snapshot restore operation is complete." - ); - - spi.stopBlock(); - - fut.get(TIMEOUT); - - client.cluster().state(ClusterState.ACTIVE); - - checkCacheKeys(client.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); - } - private void checkCacheKeys(IgniteCache testCache, int keysCnt) { assertEquals(keysCnt, testCache.size()); From 9c55d886bef40c2d0004656c0905f87ad58dac16 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 26 Jan 2021 13:37:56 +0300 Subject: [PATCH 14/98] IGNITE-13805 Don't check cache existence on finish phase (cannot raise any node-specific errors). --- .../snapshot/IgniteSnapshotManager.java | 11 ++- .../SnapshotRestoreCacheGroupProcess.java | 76 +++++++++++++------ .../snapshot/SnapshotRestoreContext.java | 42 +++++----- .../IgniteClusterSnapshoRestoreSelfTest.java | 10 ++- 4 files changed, 94 insertions(+), 45 deletions(-) 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 83c0eedc396c1c..96e33202f42c15 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 @@ -1178,9 +1178,15 @@ protected void mergeSnapshotMetadata( * @param snpName Snapshot name. * @param grpName Cache group name. * @param newFiles A list to keep track of the files created, the list updates during the restore process. + * @param interruptClosure A closure to quickly interrupt copying partition files. * @throws IgniteCheckedException If failed. */ - protected void restoreCacheGroupFiles(String snpName, String grpName, List newFiles) throws IgniteCheckedException { + protected void restoreCacheGroupFiles( + String snpName, + String grpName, + List newFiles, + Supplier interruptClosure + ) throws IgniteCheckedException { File snapshotCacheDir = resolveSnapshotCacheDir(snpName, grpName); if (!snapshotCacheDir.exists()) @@ -1207,6 +1213,9 @@ protected void restoreCacheGroupFiles(String snpName, String grpName, List } for (File snpFile : snapshotCacheDir.listFiles()) { + if (interruptClosure.get()) + return; + File target = new File(cacheDir, snpFile.getName()); if (log.isDebugEnabled()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 83ba83efd42343..254dc13cd3151e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -33,7 +33,6 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; -import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.distributed.DistributedProcess; @@ -151,6 +150,10 @@ public boolean inProgress(@Nullable String cacheName) { return !staleFuture(fut0) && (cacheName == null || fut0.context().containsCache(cacheName)); } + /** + * @param fut The future of cache snapshot restore operation. + * @return {@code True} if the future completed or not initiated. + */ public boolean staleFuture(RestoreSnapshotFuture fut) { return fut.isDone() || fut.context() == null; } @@ -270,11 +273,18 @@ private IgniteInternalFuture prepare(SnapshotRes IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); // Collect cache configuration(s). - for (String cacheName : req.groups()) { - CacheGroupSnapshotDetails grpCfg = snapshotMgr.readCacheGroupDetails(req.snapshotName(), cacheName); + for (String grpName : req.groups()) { + CacheGroupSnapshotDetails grpCfg = snapshotMgr.readCacheGroupDetails(req.snapshotName(), grpName); + + if (grpCfg == null) + continue; - if (grpCfg != null) - grpCfgs.add(grpCfg); + ensureCacheAbsent(grpName); + + for (StoredCacheData cfg : grpCfg.configs()) + ensureCacheAbsent(cfg.config().getName()); + + grpCfgs.add(grpCfg); } if (grpCfgs.isEmpty()) @@ -287,6 +297,21 @@ private IgniteInternalFuture prepare(SnapshotRes return new SnapshotRestorePrepareResponse(grpCfgs); } + /** + * Ensures that a cache with the specified name does not exist locally. + * + * @param name Cache name. + * @throws IllegalStateException If cache with the specified name already exists. + */ + private void ensureCacheAbsent(String name) throws IllegalStateException { + int id = CU.cacheId(name); + + if (ctx.cache().cacheDescriptor(id) != null || ctx.cache().cacheGroupDescriptor(id) != null) { + throw new IllegalStateException("Cache \"" + name + + "\" should be destroyed manually before perform restore operation."); + } + } + /** * Completes the verification phase and starts the restore performing phase if there were no errors. * @@ -350,13 +375,6 @@ private void finishPrepare(UUID reqId, Map opCtx.addSharedCache(cacheName, grpDetails.groupName()); cacheCfgs.add(cacheData); - - CacheGroupDescriptor desc = ctx.cache().cacheGroupDescriptor(CU.cacheId(cacheName)); - - if (desc != null) { - throw new IllegalStateException("Cache \"" + desc.cacheOrGroupName() + - "\" should be destroyed manually before perform restore operation."); - } } } @@ -449,22 +467,32 @@ private IgniteInternalFuture perform(SnapshotRes GridFutureAdapter retFut = new GridFutureAdapter<>(); - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) - return new GridFinishedFuture<>(); + try { + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) + return new GridFinishedFuture<>(); - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + for (StoredCacheData cfg : opCtx.startConfigs()) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - ctx.getSystemExecutorService().submit(() -> { - try { - opCtx.restore(updateMeta, fut0::interrupted); + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); - retFut.onDone(); - } catch (Throwable t) { - retFut.onDone(t); - } - }); + ctx.getSystemExecutorService().submit(() -> { + try { + opCtx.restore(updateMeta, fut0::interrupted); - return retFut; + retFut.onDone(); + } + catch (Throwable t) { + retFut.onDone(t); + } + }); + + return retFut; + } catch (Exception e) { + return errResponse(e); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index 36c4447b1339a3..dfe2a56b5a3d59 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -43,12 +43,6 @@ * Cache restore from snapshot operation context. */ class SnapshotRestoreContext { - private final Map> newGrpFiles = new HashMap<>(); - - private final Map pendingStartCaches = new ConcurrentHashMap<>(); - - private final Set cacheIds = new GridConcurrentHashSet<>(); - /** Request ID. */ private final UUID reqId; @@ -61,11 +55,17 @@ class SnapshotRestoreContext { /** Snapshot manager. */ private final IgniteSnapshotManager snapshotMgr; - private volatile Collection cacheCfgsToStart; - /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); + private final Map> createdFiles = new HashMap<>(); + + private final Map pendingStartCaches = new ConcurrentHashMap<>(); + + private final Set cacheIds = new GridConcurrentHashSet<>(); + + private volatile Collection cacheCfgsToStart; + /** * @param reqId Request ID. * @param snpName Snapshot name. @@ -91,14 +91,17 @@ public UUID requestId() { return reqId; } + /** @return List of baseline node IDs that must be alive to complete the operation. */ public Set nodes() { return reqNodes; } + /** @return Snapshot name. */ public String snapshotName() { return snpName; } + /** @return List of cache group names to restore from the snapshot. */ public Set groups() { return pendingStartCaches.keySet(); } @@ -125,7 +128,13 @@ public Collection startConfigs() { return cacheCfgsToStart; } - public void processCacheStart(String cacheName, @Nullable String grpName, @Nullable Throwable err, ExecutorService svc, GridFutureAdapter finishFut) { + public void processCacheStart( + String cacheName, + @Nullable String grpName, + @Nullable Throwable err, + ExecutorService svc, + GridFutureAdapter finishFut + ) { String grpName0 = grpName != null ? grpName : cacheName; PendingStartCacheGroup pendingGrp = pendingStartCaches.get(grpName0); @@ -170,8 +179,10 @@ public void processCacheStart(String cacheName, @Nullable String grpName, @Nulla } /** + * Restore specified cache groups from the local snapshot directory. + * * @param updateMetadata Update binary metadata flag. - * @param interruptClosure A closure to quickly interrupt the merge process. + * @param interruptClosure A closure to quickly interrupt the process. * @throws IgniteCheckedException If failed. */ public void restore(boolean updateMetadata, Supplier interruptClosure) throws IgniteCheckedException { @@ -185,14 +196,11 @@ public void restore(boolean updateMetadata, Supplier interruptClosure) rollbackLock.lock(); try { - if (interruptClosure.get()) - return; - List newFiles = new ArrayList<>(); - newGrpFiles.put(grpName, newFiles); + createdFiles.put(grpName, newFiles); - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles); + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles, interruptClosure); } finally { rollbackLock.unlock(); @@ -201,7 +209,7 @@ public void restore(boolean updateMetadata, Supplier interruptClosure) } /** - * Rollback changes made by process. + * Rollback changes made by process in specified cache group. * * @param grpName Cache group name. */ @@ -209,7 +217,7 @@ public void rollback(String grpName) { rollbackLock.lock(); try { - List createdFiles = newGrpFiles.remove(grpName); + List createdFiles = this.createdFiles.remove(grpName); if (F.isEmpty(createdFiles)) return; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index a90bf3c7963686..85004415ecc6bf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -49,7 +49,6 @@ import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; -import org.junit.Ignore; import org.junit.Test; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; @@ -166,7 +165,9 @@ public void testRestoreWithMissedPartitions() throws Exception { ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - putKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE, CACHE_KEYS_RANGE); + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); forceCheckpoint(); @@ -272,7 +273,7 @@ public void testRestoreSharedCacheGroup() throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(grpName, cacheName1, cacheName2)).get(TIMEOUT), + () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(TIMEOUT), IllegalArgumentException.class, "Cache group(s) not found in snapshot" ); @@ -332,7 +333,10 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { ignite0.cluster().state(ClusterState.ACTIVE); ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); + awaitPartitionMapExchange(); + ignite1.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); + awaitPartitionMapExchange(); checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); checkCacheKeys(ignite0.cache(cacheName2), CACHE_KEYS_RANGE); From 331fd757db8e37dd72a3276b8d684ce9af799dcf Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 26 Jan 2021 13:49:01 +0300 Subject: [PATCH 15/98] IGNITE-13805 Code cleanup. --- .../snapshot/CacheGroupSnapshotDetails.java | 22 +--- .../snapshot/IgniteSnapshotManager.java | 20 +-- .../SnapshotRestoreCacheGroupProcess.java | 97 +++++--------- .../snapshot/SnapshotRestoreContext.java | 120 ++++++++++-------- 4 files changed, 116 insertions(+), 143 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java index 57a2b1f13ffc2c..257dc71af3b6c9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java @@ -27,9 +27,6 @@ class CacheGroupSnapshotDetails implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; - /** Cache group name. */ - private String grpName; - /** Local partition IDs. */ private Set parts; @@ -37,34 +34,21 @@ class CacheGroupSnapshotDetails implements Serializable { private List cfgs; /** - * @param grpName Cache group name. * @param cfgs Group cache configurations. * @param parts Local partition IDs. */ - public CacheGroupSnapshotDetails(String grpName, List cfgs, Set parts) { - this.grpName = grpName; + public CacheGroupSnapshotDetails(List cfgs, Set parts) { this.cfgs = cfgs; this.parts = parts; } - /** - * @return Group cache configurations. - */ + /** @return Group cache configurations. */ public List configs() { return cfgs; } - /** - * @return Local partition IDs. - */ + /** @return Local partition IDs. */ public Set parts() { return parts; } - - /** - * @return Cache group name. - */ - public String groupName() { - return grpName; - } } 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 96e33202f42c15..38fc6d1ff66487 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 @@ -1138,7 +1138,7 @@ protected void mergeSnapshotMetadata( if (!binDir.exists()) { if (failIfAbsent) { throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + - "directory doesn't exists [snpName=" + snpName + ", dir=" + binDir + ']'); + "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); } return; @@ -1177,15 +1177,15 @@ protected void mergeSnapshotMetadata( /** * @param snpName Snapshot name. * @param grpName Cache group name. - * @param newFiles A list to keep track of the files created, the list updates during the restore process. * @param interruptClosure A closure to quickly interrupt copying partition files. + * @param newFiles A list to keep track of the files created, the list updates during the restore process. * @throws IgniteCheckedException If failed. */ protected void restoreCacheGroupFiles( String snpName, String grpName, - List newFiles, - Supplier interruptClosure + Supplier interruptClosure, + List newFiles ) throws IgniteCheckedException { File snapshotCacheDir = resolveSnapshotCacheDir(snpName, grpName); @@ -1274,25 +1274,25 @@ private File resolveSnapshotCacheDir(String snpName, String cacheName) { /** * @param snpName Snapshot name. * @param grpName Cache group name. - * @return Details about the locally stored cache group, or {@code null} if cache group (or snapshot) was not found. + * @return Details about the locally stored cache group, or {@code null} if the snapshot doesn't exist. * @throws IgniteCheckedException if failed. */ protected @Nullable CacheGroupSnapshotDetails readCacheGroupDetails( String snpName, String grpName ) throws IgniteCheckedException { - IgniteConfiguration nodeCfg = cctx.kernalContext().config(); File cacheDir = resolveSnapshotCacheDir(snpName, grpName); if (!cacheDir.exists()) return null; - List cacheCfgs = new ArrayList<>(1); - Set parts = new HashSet<>(); - + IgniteConfiguration nodeCfg = cctx.kernalContext().config(); JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(nodeCfg.getIgniteInstanceName()); ClassLoader clsLdr = U.resolveClassLoader(nodeCfg); + List cacheCfgs = new ArrayList<>(1); + Set parts = new HashSet<>(); + for (File file : cacheDir.listFiles()) { if (file.isDirectory()) continue; @@ -1313,7 +1313,7 @@ else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { } } - return new CacheGroupSnapshotDetails(grpName, cacheCfgs, parts); + return new CacheGroupSnapshotDetails(cacheCfgs, parts); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 254dc13cd3151e..24da95c6b42a05 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -30,6 +30,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.binary.BinaryObjectException; +import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -184,7 +185,7 @@ public void onNodeLeft(UUID leftNodeId) { return; if (fut0.context().nodes().contains(leftNodeId)) { - fut.onDone(new IgniteException(OP_REJECT_MSG + + fut0.onDone(new IgniteException(OP_REJECT_MSG + "Baseline node has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -195,9 +196,6 @@ public void onNodeLeft(UUID leftNodeId) { * @param reason Interruption reason. */ public void stop(String reason) { - if (ctx.clientNode()) - return; - RestoreSnapshotFuture fut0 = fut; if (staleFuture(fut0)) @@ -331,72 +329,34 @@ private void finishPrepare(UUID reqId, Map return; } - UUID updateMetadataNode = null; - - for (Map.Entry entry : res.entrySet()) { - SnapshotRestorePrepareResponse resp = entry.getValue(); - - if (resp != null && !F.isEmpty(resp.groups())) { - updateMetadataNode = entry.getKey(); - - break; - } - } - SnapshotRestoreContext opCtx = fut0.context(); - Set notFoundGroups = new HashSet<>(opCtx.groups()); + Set missedGroups = new HashSet<>(opCtx.groups()); try { - Collection grpsDetails = mergeNodeResults(res); + for (CacheGroupSnapshotDetails grpDetails : mergeNodeResults(res)) { + CacheConfiguration cfg = F.first(grpDetails.configs()).config(); - List cacheCfgs = new ArrayList<>(); + String grpName = cfg.getGroupName() == null ? cfg.getName() : cfg.getGroupName(); - for (CacheGroupSnapshotDetails grpDetails : grpsDetails) { - StoredCacheData cdata = F.first(grpDetails.configs()); - - if (cdata == null) - continue; - - int reqParts = cdata.config().getAffinity().partitions(); + int reqParts = cfg.getAffinity().partitions(); int availParts = grpDetails.parts().size(); if (reqParts != availParts) { throw new IgniteCheckedException("Cannot restore snapshot, not all partitions available [" + - "required=" + reqParts + ", avail=" + availParts + ", grp=" + grpDetails.groupName() + ']'); + "required=" + reqParts + ", avail=" + availParts + ", group=" + grpName + ']'); } - notFoundGroups.remove(grpDetails.groupName()); - - for (StoredCacheData cacheData : grpDetails.configs()) { - String cacheName = cacheData.config().getName(); - - if (!F.isEmpty(cacheData.config().getGroupName())) - opCtx.addSharedCache(cacheName, grpDetails.groupName()); + missedGroups.remove(grpName); - cacheCfgs.add(cacheData); - } + for (StoredCacheData cacheData : grpDetails.configs()) + opCtx.addCacheData(cacheData); } - if (!notFoundGroups.isEmpty()) { + if (!missedGroups.isEmpty()) { throw new IllegalArgumentException("Cache group(s) not found in snapshot [groups=" + - F.concat(notFoundGroups, ", ") + ", snapshot=" + opCtx.snapshotName() + ']'); + F.concat(missedGroups, ", ") + ", snapshot=" + opCtx.snapshotName() + ']'); } - - Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - F.node2id(), - (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - - Set reqNodes = new HashSet<>(opCtx.nodes()); - - reqNodes.removeAll(srvNodeIds); - - if (!reqNodes.isEmpty()) { - throw new IllegalStateException("Unable to perform a restore operation, server node(s) left " + - "the cluster [nodeIds=" + F.concat(reqNodes, ", ") + ']'); - } - - opCtx.startConfigs(cacheCfgs); } catch (Exception e) { fut0.onDone(e); @@ -404,8 +364,11 @@ private void finishPrepare(UUID reqId, Map return; } - if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) - performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, updateMetadataNode)); + if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) { + UUID metaUpdateNode = F.first(F.viewReadOnly(res.entrySet(), Map.Entry::getKey, e -> e.getValue() != null)); + + performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, metaUpdateNode)); + } } /** @@ -417,15 +380,17 @@ private Collection mergeNodeResults(Map entry : res.entrySet()) { UUID currNodeId = entry.getKey(); - SnapshotRestorePrepareResponse singleResp = entry.getValue(); + SnapshotRestorePrepareResponse nodeResp = entry.getValue(); - if (singleResp == null) + if (nodeResp == null) continue; - for (CacheGroupSnapshotDetails nodeDetails : singleResp.groups()) { - T2 clusterDetailsPair = globalDetails.get(nodeDetails.groupName()); + for (CacheGroupSnapshotDetails nodeDetails : nodeResp.groups()) { + CacheConfiguration cfg = F.first(nodeDetails.configs()).config(); - String grpName = nodeDetails.groupName(); + String grpName = cfg.getGroupName() == null ? cfg.getName() : cfg.getGroupName(); + + T2 clusterDetailsPair = globalDetails.get(grpName); if (clusterDetailsPair == null) { globalDetails.put(grpName, new T2<>(currNodeId, nodeDetails)); @@ -460,6 +425,10 @@ private IgniteInternalFuture perform(SnapshotRes return new GridFinishedFuture<>(); RestoreSnapshotFuture fut0 = fut; + + if (fut0.isDone() || fut0.interrupted()) + return new GridFinishedFuture<>(); + SnapshotRestoreContext opCtx = fut0.context(); if (!req.requestId().equals(opCtx.requestId())) @@ -471,7 +440,7 @@ private IgniteInternalFuture perform(SnapshotRes if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) return new GridFinishedFuture<>(); - for (StoredCacheData cfg : opCtx.startConfigs()) { + for (StoredCacheData cfg : opCtx.configs()) { if (!F.isEmpty(cfg.config().getGroupName())) ensureCacheAbsent(cfg.config().getName()); } @@ -517,7 +486,7 @@ private void finishPerform(UUID reqId, Map if (!U.isLocalNodeCoordinator(ctx.discovery())) return; - ctx.cache().dynamicStartCachesByStoredConf(fut0.context().startConfigs(), true, true, false, null, true); + ctx.cache().dynamicStartCachesByStoredConf(fut0.context().configs(), true, true, false, null, true); } /** @@ -548,14 +517,14 @@ private class RestoreSnapshotFuture extends GridFutureAdapter { private volatile SnapshotRestoreContext ctx; /** - * @return Snapshot restore operation context. + * @return Cache group restore from snapshot operation context. */ public SnapshotRestoreContext context() { return ctx; } /** - * @param ctx Snapshot restore operation context. + * @param ctx Cache group restore from snapshot operation context. */ public void init(SnapshotRestoreContext ctx) { this.ctx = ctx; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index dfe2a56b5a3d59..81e853ffa95721 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -21,7 +21,7 @@ import java.io.File; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -40,7 +40,7 @@ import org.jetbrains.annotations.Nullable; /** - * Cache restore from snapshot operation context. + * Cache group restore from snapshot operation context. */ class SnapshotRestoreContext { /** Request ID. */ @@ -58,13 +58,11 @@ class SnapshotRestoreContext { /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); - private final Map> createdFiles = new HashMap<>(); + /** Cache configurations. */ + private final Map cacheCfgs = new ConcurrentHashMap<>(); - private final Map pendingStartCaches = new ConcurrentHashMap<>(); - - private final Set cacheIds = new GridConcurrentHashSet<>(); - - private volatile Collection cacheCfgsToStart; + /** Restored cache groups. */ + private final Map grps = new ConcurrentHashMap<>(); /** * @param reqId Request ID. @@ -75,10 +73,8 @@ class SnapshotRestoreContext { */ public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Collection grps, IgniteSnapshotManager snapshotMgr) { - for (String grpName : grps) { - cacheIds.add(CU.cacheId(grpName)); - pendingStartCaches.put(grpName, new PendingStartCacheGroup()); - } + for (String grpName : grps) + this.grps.put(grpName, new GroupRestoreContext()); this.reqId = reqId; this.reqNodes = reqNodes; @@ -93,7 +89,7 @@ public UUID requestId() { /** @return List of baseline node IDs that must be alive to complete the operation. */ public Set nodes() { - return reqNodes; + return Collections.unmodifiableSet(reqNodes); } /** @return Snapshot name. */ @@ -103,31 +99,49 @@ public String snapshotName() { /** @return List of cache group names to restore from the snapshot. */ public Set groups() { - return pendingStartCaches.keySet(); + return grps.keySet(); } - public void addSharedCache(String cacheName, String grpName) { - cacheIds.add(CU.cacheId(cacheName)); + /** + * @param name Cache name. + * @return {@code True} if the cache with the specified name is currently being restored. + */ + public boolean containsCache(String name) { + return grps.containsKey(name) || cacheCfgs.containsKey(CU.cacheId(name)); + } - PendingStartCacheGroup sharedGrp = pendingStartCaches.get(grpName); + /** @return Cache configurations. */ + public Collection configs() { + return cacheCfgs.values(); + } - assert sharedGrp != null : grpName; + /** + * @param cacheData Stored cache data. + */ + public void addCacheData(StoredCacheData cacheData) { + String cacheName = cacheData.config().getName(); - sharedGrp.caches.add(cacheName); - } + cacheCfgs.put(CU.cacheId(cacheName), cacheData); - public boolean containsCache(String name) { - return cacheIds.contains(CU.cacheId(name)); - } + String grpName = cacheData.config().getGroupName(); - public void startConfigs(Collection ccfgs) { - cacheCfgsToStart = ccfgs; - } + if (grpName == null) + return; + + GroupRestoreContext grpCtx = grps.get(grpName); + + assert grpCtx != null : grpName; - public Collection startConfigs() { - return cacheCfgsToStart; + grpCtx.caches.add(cacheName); } + /** + * @param cacheName Cache name. + * @param grpName Group name. + * @param err Exception (if any). + * @param svc Executor service for asynchronous rollback. + * @param finishFut A future to be completed when all restored cache groups are started or rolled back. + */ public void processCacheStart( String cacheName, @Nullable String grpName, @@ -137,25 +151,27 @@ public void processCacheStart( ) { String grpName0 = grpName != null ? grpName : cacheName; - PendingStartCacheGroup pendingGrp = pendingStartCaches.get(grpName0); + GroupRestoreContext grp = grps.get(grpName0); // If any of shared caches has been started - we cannot rollback changes. - if (pendingGrp.caches.remove(cacheName) && err == null) - pendingGrp.canRollback = false; + if (grp.caches.remove(cacheName) && err == null) + grp.started = true; + + if (!grp.caches.isEmpty()) { + if (err != null) + grp.startErr = err; - if (!pendingGrp.caches.isEmpty()) return; + } - if (err != null && pendingGrp.canRollback) { + if (err != null && !grp.started) { svc.submit(() -> { rollbackLock.lock(); try { - pendingStartCaches.remove(grpName0); - rollback(grpName0); - if (pendingStartCaches.isEmpty()) + if (grps.isEmpty()) finishFut.onDone(err); } finally { @@ -169,10 +185,8 @@ public void processCacheStart( rollbackLock.lock(); try { - pendingStartCaches.remove(grpName0); - - if (pendingStartCaches.isEmpty()) - finishFut.onDone(); + if (grps.remove(grpName0) != null && grps.isEmpty()) + finishFut.onDone(null, err == null ? grp.startErr : err); } finally { rollbackLock.unlock(); } @@ -196,11 +210,9 @@ public void restore(boolean updateMetadata, Supplier interruptClosure) rollbackLock.lock(); try { - List newFiles = new ArrayList<>(); + GroupRestoreContext grp = grps.get(grpName); - createdFiles.put(grpName, newFiles); - - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, newFiles, interruptClosure); + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, interruptClosure, grp.files); } finally { rollbackLock.unlock(); @@ -217,12 +229,12 @@ public void rollback(String grpName) { rollbackLock.lock(); try { - List createdFiles = this.createdFiles.remove(grpName); + GroupRestoreContext grp = grps.remove(grpName); - if (F.isEmpty(createdFiles)) + if (grp == null || F.isEmpty(grp.files)) return; - snapshotMgr.rollbackRestoreOperation(createdFiles); + snapshotMgr.rollbackRestoreOperation(grp.files); } finally { rollbackLock.unlock(); } @@ -234,9 +246,17 @@ public void rollback(String grpName) { } /** */ - private static class PendingStartCacheGroup { - volatile boolean canRollback = true; + private static class GroupRestoreContext { + /** List of caches of the cache group. */ + final Set caches = new GridConcurrentHashSet<>(); + + /** Files created in the cache group folder during a restore operation. */ + final List files = new ArrayList<>(); + + /** The flag indicates that one of the caches in this cache group has been started. */ + volatile boolean started; - Set caches = new GridConcurrentHashSet<>(); + /** An exception that was thrown when starting a shared cache group (if any). */ + volatile Throwable startErr; } } From ad549f6eeb9145fae7f84ac058854112866c0bae Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 26 Jan 2021 18:36:24 +0300 Subject: [PATCH 16/98] IGNITE-13805 Minor code cleanup. --- .../snapshot/IgniteSnapshotManager.java | 2 +- .../IgniteClusterSnapshoRestoreSelfTest.java | 173 +++++++++++++----- 2 files changed, 132 insertions(+), 43 deletions(-) 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 38fc6d1ff66487..d1fc3e77c7491f 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 @@ -1234,7 +1234,7 @@ protected void restoreCacheGroupFiles( } /** - * @param files List pf created files. + * @param files Collection of files to delete. */ protected void rollbackRestoreOperation(Collection files) { List dirs = new ArrayList<>(); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index 85004415ecc6bf..ccd2ead469a784 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -65,11 +65,14 @@ public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTes /** Timeout. */ private static final long TIMEOUT = 15_000; + /** Binary type name. */ private static final String BIN_TYPE_NAME = "customType"; - protected CacheConfiguration[] cacheCfgs; + /** Static cache configurations. */ + protected CacheConfiguration[] cacheCfgs; - protected Function valueBuilder = new IndexedValueBuilder(); + /** Cache value builder. */ + protected Function valBuilder = new IndexedValueBuilder(); /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { @@ -86,6 +89,9 @@ public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTes return cfg; } + /** + * @param typeName Type name. + */ private QueryEntity queryEntity(String typeName) { return new QueryEntity() .setKeyType("java.lang.Integer") @@ -94,12 +100,7 @@ private QueryEntity queryEntity(String typeName) { .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); } - /** {@inheritDoc} */ - @Override public void afterTestSnapshot() throws Exception { - stopAllGrids(); - } - - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testBasicClusterSnapshotRestore() throws Exception { int keysCnt = 10_000; @@ -116,12 +117,12 @@ public void testBasicClusterSnapshotRestore() throws Exception { checkCacheKeys(cache, keysCnt); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { int keysCnt = 10_000; - valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); @@ -142,10 +143,10 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { checkCacheKeys(cache, keysCnt); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception { - IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg); + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, dfltCacheCfg); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); @@ -158,10 +159,10 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testRestoreWithMissedPartitions() throws Exception { - IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valueBuilder, dfltCacheCfg.setBackups(0)); + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, dfltCacheCfg.setBackups(0)); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); @@ -198,16 +199,16 @@ public void testRestoreWithMissedPartitions() throws Exception { checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testClusterSnapshotRestoreDiffTopology() throws Exception { int nodesCnt = 4; int keysCnt = 10_000; - valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); - IgniteEx ignite = startGridsWithCache(nodesCnt - 2, keysCnt, valueBuilder, dfltCacheCfg); + IgniteEx ignite = startGridsWithCache(nodesCnt - 2, keysCnt, valBuilder, dfltCacheCfg); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); @@ -241,7 +242,7 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { checkCacheKeys(cache, keysCnt); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testRestoreSharedCacheGroup() throws Exception { String grpName = "shared"; @@ -288,7 +289,7 @@ public void testRestoreSharedCacheGroup() throws Exception { checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testRestoreCacheGroupWithNodeFilter() throws Exception { String cacheName1 = "cache1"; @@ -342,10 +343,10 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { checkCacheKeys(ignite0.cache(cacheName2), CACHE_KEYS_RANGE); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testIncompatibleMetasUpdate() throws Exception { - valueBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); @@ -411,7 +412,17 @@ public void testIncompatibleMetasUpdate() throws Exception { assertEquals(objs[i], cache1.get(i)); } - private IgniteCache createCacheWithBinaryType(Ignite ignite, String cacheName, Function valBuilder) { + /** + * @param ignite Ignite. + * @param cacheName Cache name. + * @param valBuilder Binary value builder. + * @return Created cache. + */ + private IgniteCache createCacheWithBinaryType( + Ignite ignite, + String cacheName, + Function valBuilder + ) { IgniteCache cache = ignite.createCache(new CacheConfiguration<>(cacheName)).withKeepBinary(); for (int i = 0; i < CACHE_KEYS_RANGE; i++) @@ -420,16 +431,26 @@ private IgniteCache createCacheWithBinaryType(Ignite ignite, St return cache; } + /** + * @throws Exception if failed + */ @Test public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { checkCacheStartWithTheSameName(true); } + /** + * @throws Exception if failed + */ @Test public void testParallelCacheStartWithTheSameNameOnPerform() throws Exception { checkCacheStartWithTheSameName(false); } + /** + * @param prepare {@code True} to start cache during prepare phase, {@code False} to start cache during perform phase. + * @throws Exception if failed. + */ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { String grpName = "shared"; String cacheName = "cache1"; @@ -440,7 +461,8 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE : RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, grpName); + IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? + RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE : RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, grpName); String msgFormat = "Cache start failed. A cache named \"%s\" is currently being restored from a snapshot."; @@ -483,18 +505,22 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { } } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testRollbackOnNodeFail() throws Exception { checkBaselineChange(true); } - /** @throws Exception If fails. */ + /** @throws Exception If failed. */ @Test public void testNodeJoin() throws Exception { checkBaselineChange(false); } + /** + * @param stopNode {@code True} to check node fail, {@code False} to check node join. + * @throws Exception if failed. + */ private void checkBaselineChange(boolean stopNode) throws Exception { int keysCnt = 10_000; @@ -537,37 +563,70 @@ private void checkBaselineChange(boolean stopNode) throws Exception { checkCacheKeys(cache, keysCnt); } + /** + * @throws Exception if failed. + */ @Test public void testClusterStateChangeActiveReadonlyDuringPrepare() throws Exception { checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); } + + /** + * @throws Exception if failed. + */ @Test public void testClusterStateChangeActiveReadonlyDuringPerform() throws Exception { checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); } + /** + * @param procType The type of distributed process on which communication is blocked. + * @throws Exception if failed. + */ private void checkReadOnlyDuringRestoring(DistributedProcessType procType) throws Exception { checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, procType, IgniteClusterReadOnlyException.class, "Failed to perform start cache operation (cluster is in read-only mode)"); } + /** + * @throws Exception if failed. + */ @Test public void testClusterDeactivateOnPrepare() throws Exception { checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); } + /** + * @throws Exception if failed. + */ @Test public void testClusterDeactivateOnPerform() throws Exception { checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); } + /** + * @param procType The type of distributed process on which communication is blocked. + * @throws Exception if failed. + */ private void checkDeactivationDuringRestoring(DistributedProcessType procType) throws Exception { checkClusterStateChange(ClusterState.INACTIVE, procType, IgniteCheckedException.class, "The cluster has been deactivated."); } - private void checkClusterStateChange(ClusterState state, DistributedProcessType procType, Class expCls, String expMsg) throws Exception { + /** + * @param state Cluster state. + * @param procType The type of distributed process on which communication is blocked. + * @param expCls Expected exception class. + * @param expMsg Expected exception message. + * @throws Exception if failed. + */ + private void checkClusterStateChange( + ClusterState state, + DistributedProcessType procType, + Class expCls, + String expMsg + ) throws Exception { Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); @@ -589,6 +648,11 @@ private void checkClusterStateChange(ClusterState state, DistributedProcessType checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } + /** + * @param nodesCnt Count of nodes. + * @param cacheName Cache name. + * @throws IgniteCheckedException if failed. + */ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCheckedException { for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { IgniteEx grid = grid(nodeIdx); @@ -602,6 +666,12 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh } } + /** + * @param ignite Ignite. + * @param cacheOrGrpName Cache (or group) name. + * @return Local path to the cache directory. + * @throws IgniteCheckedException if failed. + */ private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws IgniteCheckedException { File workDIr = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); @@ -615,8 +685,14 @@ private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws Igni return new File(workDIr, nodeDirName + CACHE_GRP_DIR_PREFIX + cacheOrGrpName); } + /** + * @param nodesCnt Nodes count. + * @param keysCnt Number of keys to create. + * @return Ignite coordinator instance. + * @throws Exception if failed. + */ private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { - IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder, dfltCacheCfg); + IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valBuilder, dfltCacheCfg); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); @@ -627,7 +703,18 @@ private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Except return ignite; } - private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi spi, DistributedProcessType restorePhase, String grpName) throws InterruptedException { + /** + * @param spi Test communication spi. + * @param restorePhase The type of distributed process on which communication is blocked. + * @param grpName Cache group name. + * @return Snapshot restore future. + * @throws InterruptedException if interrupted. + */ + private IgniteFuture waitForBlockOnRestore( + TestRecordingCommunicationSpi spi, + DistributedProcessType restorePhase, + String grpName + ) throws InterruptedException { spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == restorePhase.ordinal()); @@ -639,26 +726,28 @@ private IgniteFuture waitForBlockOnRestore(TestRecordingCommunicationSpi s return fut; } - private void checkCacheKeys(IgniteCache testCache, int keysCnt) { - assertEquals(keysCnt, testCache.size()); - - for (int i = 0; i < keysCnt; i++) - assertEquals(valueBuilder.apply(i), testCache.get(i)); - } - + /** + * @param cache Cache. + * @param startIdx The initial value of the number for the key. + * @param cnt Number of entries to put in the cache. + */ private void putKeys(IgniteCache cache, int startIdx, int cnt) { for (int i = startIdx; i < (startIdx + cnt); i++) - cache.put(i, valueBuilder.apply(i)); + cache.put(i, valBuilder.apply(i)); } - /** */ - private static class IntValueBuilder implements Function { - /** {@inheritDoc} */ - @Override public Object apply(Integer key) { - return key; - } + /** + * @param cache Cache. + * @param keysCnt Expected number of keys. + */ + private void checkCacheKeys(IgniteCache cache, int keysCnt) { + assertEquals(keysCnt, cache.size()); + + for (int i = 0; i < keysCnt; i++) + assertEquals(valBuilder.apply(i), cache.get(i)); } + /** */ private static class IndexedValueBuilder implements Function { /** {@inheritDoc} */ @Override public Object apply(Integer key) { From 3959cecc44bedaa2aa54f0616d6bf356640bd4bb Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 27 Jan 2021 16:49:23 +0300 Subject: [PATCH 17/98] IGNITE-13805 Minor code cleanup. --- .../snapshot/IgniteSnapshotManager.java | 23 +++++++++++-------- .../snapshot/SnapshotRestoreContext.java | 12 +++++----- 2 files changed, 19 insertions(+), 16 deletions(-) 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 d1fc3e77c7491f..827da848b68e38 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 @@ -55,8 +55,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; +import java.util.function.BooleanSupplier; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -1036,7 +1036,7 @@ public List readSnapshotMetadatas(String snpName) { if (isCacheRestoring(null)) { throw new IgniteException("Snapshot operation has been rejected. " + - "Cache group restore operation is currently in prgoress."); + "Cache group restore operation is currently in progress."); } if (cctx.kernalContext().clientNode()) { @@ -1131,7 +1131,7 @@ protected void mergeSnapshotMetadata( String snpName, boolean checkCompatibility, boolean failIfAbsent, - Supplier interruptClosure + BooleanSupplier interruptClosure ) throws IgniteCheckedException { File binDir = binaryWorkDir(snapshotLocalDir(snpName).getAbsolutePath(), pdsSettings.folderName()); @@ -1149,7 +1149,7 @@ protected void mergeSnapshotMetadata( CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); for (File file : binDir.listFiles()) { - if (interruptClosure.get()) + if (interruptClosure.getAsBoolean()) return; try (FileInputStream in = new FileInputStream(file)) { @@ -1177,14 +1177,14 @@ protected void mergeSnapshotMetadata( /** * @param snpName Snapshot name. * @param grpName Cache group name. - * @param interruptClosure A closure to quickly interrupt copying partition files. + * @param stopChecker Node stop or prcoess interrupt checker. * @param newFiles A list to keep track of the files created, the list updates during the restore process. * @throws IgniteCheckedException If failed. */ protected void restoreCacheGroupFiles( String snpName, String grpName, - Supplier interruptClosure, + BooleanSupplier stopChecker, List newFiles ) throws IgniteCheckedException { File snapshotCacheDir = resolveSnapshotCacheDir(snpName, grpName); @@ -1213,7 +1213,7 @@ protected void restoreCacheGroupFiles( } for (File snpFile : snapshotCacheDir.listFiles()) { - if (interruptClosure.get()) + if (stopChecker.getAsBoolean()) return; File target = new File(cacheDir, snpFile.getName()); @@ -1246,11 +1246,14 @@ protected void rollbackRestoreOperation(Collection files) { if (file.isDirectory()) dirs.add(file); - file.delete(); + if (!file.delete()) + log.warning("Unable to delete a file created during a cache restore operation [file=" + file + ']'); } - for (File dir : dirs) - dir.delete(); + for (File dir : dirs) { + if (!dir.delete()) + log.warning("Unable to delete a folder created during a cache restore operation [file=" + dir + ']'); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index 81e853ffa95721..bcba1ba5b4ae02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -29,7 +29,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Supplier; +import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.GridConcurrentHashSet; @@ -196,15 +196,15 @@ public void processCacheStart( * Restore specified cache groups from the local snapshot directory. * * @param updateMetadata Update binary metadata flag. - * @param interruptClosure A closure to quickly interrupt the process. + * @param stopChecker Node stop or prcoess interrupt checker. * @throws IgniteCheckedException If failed. */ - public void restore(boolean updateMetadata, Supplier interruptClosure) throws IgniteCheckedException { - if (interruptClosure.get()) + public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws IgniteCheckedException { + if (stopChecker.getAsBoolean()) return; if (updateMetadata) - snapshotMgr.mergeSnapshotMetadata(snpName, false, true, interruptClosure); + snapshotMgr.mergeSnapshotMetadata(snpName, false, true, stopChecker); for (String grpName : groups()) { rollbackLock.lock(); @@ -212,7 +212,7 @@ public void restore(boolean updateMetadata, Supplier interruptClosure) try { GroupRestoreContext grp = grps.get(grpName); - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, interruptClosure, grp.files); + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, stopChecker, grp.files); } finally { rollbackLock.unlock(); From 622bc9db036cdd708ecb65901d469b462bd61eb2 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 27 Jan 2021 16:58:06 +0300 Subject: [PATCH 18/98] IGNITE-13805 Minor code cleanup. --- .../snapshot/CacheGroupSnapshotDetails.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java index 257dc71af3b6c9..64e2da66676bd2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java @@ -22,7 +22,9 @@ import java.util.Set; import org.apache.ignite.internal.processors.cache.StoredCacheData; -/** */ +/** + * Contains information about partitions and cache configurations located in the cache group snapshot directory. + */ class CacheGroupSnapshotDetails implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -30,11 +32,11 @@ class CacheGroupSnapshotDetails implements Serializable { /** Local partition IDs. */ private Set parts; - /** Group cache configurations. */ + /** Stored cache configurations. */ private List cfgs; /** - * @param cfgs Group cache configurations. + * @param cfgs Stored cache configurations. * @param parts Local partition IDs. */ public CacheGroupSnapshotDetails(List cfgs, Set parts) { @@ -42,7 +44,7 @@ public CacheGroupSnapshotDetails(List cfgs, Set parts) this.parts = parts; } - /** @return Group cache configurations. */ + /** @return Stored cache configurations. */ public List configs() { return cfgs; } From 8271bb5b5f952cb4a675d563ff14908ca0051d61 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 27 Jan 2021 17:29:04 +0300 Subject: [PATCH 19/98] IGNITE-13805 Minor code cleanup. --- .../SnapshotRestoreCacheGroupProcess.java | 40 +++++-------------- 1 file changed, 11 insertions(+), 29 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 24da95c6b42a05..7d3ad8a5cbe79b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -214,11 +214,13 @@ private IgniteInternalFuture prepare(SnapshotRes if (ctx.clientNode()) return new GridFinishedFuture<>(); - if (inProgress(null)) - return errResponse(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); + if (inProgress(null)) { + return new GridFinishedFuture<>( + new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); + } if (!ctx.state().clusterState().state().active()) - return errResponse(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); + return new GridFinishedFuture<>(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); // Skip creating future on initiator. if (fut.isDone()) @@ -320,7 +322,7 @@ private void ensureCacheAbsent(String name) throws IllegalStateException { private void finishPrepare(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (fut0.interrupted() || !reqId.equals(fut0.context().requestId()) || fut0.isDone()) + if (fut0.isDone() || fut0.interrupted() || !reqId.equals(fut0.context().requestId())) return; if (!errs.isEmpty()) { @@ -421,9 +423,6 @@ private Collection mergeNodeResults(Map perform(SnapshotRestorePerformRequest req) { - if (ctx.clientNode()) - return new GridFinishedFuture<>(); - RestoreSnapshotFuture fut0 = fut; if (fut0.isDone() || fut0.interrupted()) @@ -432,7 +431,7 @@ private IgniteInternalFuture perform(SnapshotRes SnapshotRestoreContext opCtx = fut0.context(); if (!req.requestId().equals(opCtx.requestId())) - return errResponse("Unknown snapshot restore operation was rejected."); + return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -460,7 +459,7 @@ private IgniteInternalFuture perform(SnapshotRes return retFut; } catch (Exception e) { - return errResponse(e); + return new GridFinishedFuture<>(e); } } @@ -472,7 +471,7 @@ private IgniteInternalFuture perform(SnapshotRes private void finishPerform(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.interrupted()) + if (fut0.isDone() || fut0.interrupted() || !reqId.equals(fut0.context().requestId())) return; Exception failure = F.first(errs.values()); @@ -489,24 +488,6 @@ private void finishPerform(UUID reqId, Map ctx.cache().dynamicStartCachesByStoredConf(fut0.context().configs(), true, true, false, null, true); } - /** - * @param msg Error message. - * @param Type of the future. - * @return Failed with the specified error message future. - */ - private IgniteInternalFuture errResponse(String msg) { - return errResponse(new IgniteException(msg)); - } - - /** - * @param ex Exception. - * @param Type of the future. - * @return Failed with the specified exception future. - */ - private IgniteInternalFuture errResponse(Exception ex) { - return new GridFinishedFuture<>(ex); - } - /** */ private class RestoreSnapshotFuture extends GridFutureAdapter { /** The exception that led to the interruption of the process. */ @@ -547,7 +528,8 @@ public boolean interrupted() { Set grpNames = opCtx0.groups(); - log.error("Snapshot restore process has been interrupted [grps=" + grpNames + ']', err); + log.error("Snapshot restore process has been interrupted " + + "[groups=" + grpNames + ", snapshot=" + opCtx0.snapshotName() + ']', err); for (String grpName : grpNames) opCtx0.rollback(grpName); From 28975ea7ada60c82fa114fa522eed65d8bee92a8 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 28 Jan 2021 12:43:24 +0300 Subject: [PATCH 20/98] IGNITE-13805 Read cache configs etc should be in filepagestore manager. --- .../file/FilePageStoreManager.java | 62 +++++--- .../snapshot/CacheGroupSnapshotDetails.java | 56 ------- .../snapshot/IgniteSnapshotManager.java | 50 +----- .../SnapshotRestoreCacheGroupProcess.java | 142 ++++++++---------- .../SnapshotRestorePrepareResponse.java | 21 ++- .../IgniteClusterSnapshoRestoreSelfTest.java | 4 + 6 files changed, 132 insertions(+), 203 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java index a7c682dc320138..2611234cd636f3 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java @@ -38,6 +38,7 @@ 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; @@ -974,29 +975,37 @@ public void readConfigurationFiles(List> ccfgs, Arrays.sort(files); for (File file : files) { - if (file.isDirectory()) { - if (file.getName().startsWith(CACHE_DIR_PREFIX)) { - File conf = new File(file, CACHE_DATA_FILENAME); + if (file.isDirectory()) + readCacheConfigurations(file, ccfgs); + } + + return ccfgs; + } + + /** + * @param dir Cache (group) directory. + * @param ccfgs Cache configurations. + * @throws IgniteCheckedException If failed. + */ + public void readCacheConfigurations(File dir, Map ccfgs) throws IgniteCheckedException { + if (dir.getName().startsWith(CACHE_DIR_PREFIX)) { + File conf = new File(dir, CACHE_DATA_FILENAME); - if (conf.exists() && conf.length() > 0) { - StoredCacheData cacheData = readCacheData(conf); + if (conf.exists() && conf.length() > 0) { + StoredCacheData cacheData = readCacheData(conf); - String cacheName = cacheData.config().getName(); + String cacheName = cacheData.config().getName(); - if (!ccfgs.containsKey(cacheName)) - ccfgs.put(cacheName, cacheData); - else { - U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file " - + file.getName()); - } - } + if (!ccfgs.containsKey(cacheName)) + ccfgs.put(cacheName, cacheData); + else { + U.warn(log, "Cache with name=" + cacheName + " is already registered, skipping config file " + + dir.getName()); } - else if (file.getName().startsWith(CACHE_GRP_DIR_PREFIX)) - readCacheGroupCaches(file, ccfgs); } } - - return ccfgs; + else if (dir.getName().startsWith(CACHE_GRP_DIR_PREFIX)) + readCacheGroupCaches(dir, ccfgs); } /** @@ -1089,6 +1098,25 @@ private void readCacheGroupCaches(File grpDir, Map ccfg } } + /** + * @param cacheDIr Directory with partition files. + * @return Set of partition file IDs found in the directory. + */ + public Set scanPartitionIds(File cacheDIr) { + Set partIds = new HashSet<>(); + + for (String name : cacheDIr.list((dir, name) -> name.startsWith(PART_FILE_PREFIX))) { + if (new File(cacheDIr, name).isDirectory()) + continue; + + String partId = name.substring(PART_FILE_PREFIX.length(), name.indexOf('.')); + + partIds.add(Integer.parseInt(partId)); + } + + return partIds; + } + /** * @param conf File with stored cache data. * @return Cache data. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java deleted file mode 100644 index 64e2da66676bd2..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/CacheGroupSnapshotDetails.java +++ /dev/null @@ -1,56 +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.Serializable; -import java.util.List; -import java.util.Set; -import org.apache.ignite.internal.processors.cache.StoredCacheData; - -/** - * Contains information about partitions and cache configurations located in the cache group snapshot directory. - */ -class CacheGroupSnapshotDetails implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Local partition IDs. */ - private Set parts; - - /** Stored cache configurations. */ - private List cfgs; - - /** - * @param cfgs Stored cache configurations. - * @param parts Local partition IDs. - */ - public CacheGroupSnapshotDetails(List cfgs, Set parts) { - this.cfgs = cfgs; - this.parts = parts; - } - - /** @return Stored cache configurations. */ - public List configs() { - return cfgs; - } - - /** @return Local partition IDs. */ - public Set parts() { - return parts; - } -} 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 827da848b68e38..dd6a9b1aeb7e5d 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 @@ -136,8 +136,6 @@ import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.marshaller.Marshaller; -import org.apache.ignite.marshaller.MarshallerUtils; -import org.apache.ignite.marshaller.jdk.JdkMarshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.apache.ignite.thread.OomExceptionHandler; @@ -167,7 +165,6 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.pageIndex; import static org.apache.ignite.internal.pagemem.PageIdUtils.toDetailString; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DATA_FILENAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; @@ -1261,7 +1258,7 @@ protected void rollbackRestoreOperation(Collection files) { * @param cacheName Cache (group) name. * @return Local path to the cache directory. */ - private File resolveSnapshotCacheDir(String snpName, String cacheName) { + public File resolveSnapshotCacheDir(String snpName, String cacheName) { File workDir = snapshotLocalDir(snpName); String dbPath = DFLT_STORE_DIR + File.separator + pdsSettings.folderName() + File.separator; @@ -1274,51 +1271,6 @@ private File resolveSnapshotCacheDir(String snpName, String cacheName) { return new File(workDir, dbPath + CACHE_GRP_DIR_PREFIX + cacheName); } - /** - * @param snpName Snapshot name. - * @param grpName Cache group name. - * @return Details about the locally stored cache group, or {@code null} if the snapshot doesn't exist. - * @throws IgniteCheckedException if failed. - */ - protected @Nullable CacheGroupSnapshotDetails readCacheGroupDetails( - String snpName, - String grpName - ) throws IgniteCheckedException { - File cacheDir = resolveSnapshotCacheDir(snpName, grpName); - - if (!cacheDir.exists()) - return null; - - IgniteConfiguration nodeCfg = cctx.kernalContext().config(); - JdkMarshaller marshaller = MarshallerUtils.jdkMarshaller(nodeCfg.getIgniteInstanceName()); - ClassLoader clsLdr = U.resolveClassLoader(nodeCfg); - - List cacheCfgs = new ArrayList<>(1); - Set parts = new HashSet<>(); - - for (File file : cacheDir.listFiles()) { - if (file.isDirectory()) - continue; - - String name = file.getName(); - - if (name.endsWith(CACHE_DATA_FILENAME) && file.length() > 0) { - try (InputStream stream = new BufferedInputStream(new FileInputStream(file))) { - cacheCfgs.add(marshaller.unmarshal(stream, clsLdr)); - } catch (IOException e) { - throw new IgniteCheckedException("Unable to read stored cache configuration: " + e.getMessage(), e); - } - } - else if (name.startsWith(FilePageStoreManager.PART_FILE_PREFIX)) { - String partId = name.substring(FilePageStoreManager.PART_FILE_PREFIX.length(), name.indexOf('.')); - - parts.add(Integer.parseInt(partId)); - } - } - - return new CacheGroupSnapshotDetails(cacheCfgs, parts); - } - /** {@inheritDoc} */ @Override public void onReadyForReadWrite(ReadWriteMetastorage metaStorage) throws IgniteCheckedException { synchronized (snpOpMux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 7d3ad8a5cbe79b..e61014a1d7ca22 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; +import java.io.File; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -35,6 +36,7 @@ import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -43,11 +45,9 @@ import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; import org.apache.ignite.internal.util.typedef.internal.U; -import org.apache.ignite.lang.IgniteBiTuple; import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; @@ -268,33 +268,40 @@ private IgniteInternalFuture prepare(SnapshotRes if (log.isInfoEnabled()) log.info("Preparing to restore cache groups [groups=" + F.concat(req.groups(), ", ") + ']'); - List grpCfgs = new ArrayList<>(); - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + Map cacheCfgs = new HashMap<>(); + Map> partIds = new HashMap<>(); + // Collect cache configuration(s). for (String grpName : req.groups()) { - CacheGroupSnapshotDetails grpCfg = snapshotMgr.readCacheGroupDetails(req.snapshotName(), grpName); + File cacheDir = snapshotMgr.resolveSnapshotCacheDir(req.snapshotName(), grpName); - if (grpCfg == null) - continue; + if (!cacheDir.exists()) + return null; - ensureCacheAbsent(grpName); + FilePageStoreManager pageStoreMgr = (FilePageStoreManager)ctx.cache().context().pageStore(); - for (StoredCacheData cfg : grpCfg.configs()) - ensureCacheAbsent(cfg.config().getName()); + pageStoreMgr.readCacheConfigurations(cacheDir, cacheCfgs); - grpCfgs.add(grpCfg); + partIds.put(grpName, pageStoreMgr.scanPartitionIds(cacheDir)); } - if (grpCfgs.isEmpty()) + if (cacheCfgs.isEmpty()) return null; + Set cacheNames = new HashSet<>(req.groups()); + + cacheNames.addAll(cacheCfgs.keySet()); + + for (String cacheName : cacheNames) + ensureCacheAbsent(cacheName); + RestoreSnapshotFuture fut0 = fut; ctx.cache().context().snapshotMgr().mergeSnapshotMetadata(req.snapshotName(), true, false, fut0::interrupted); - return new SnapshotRestorePrepareResponse(grpCfgs); + return new SnapshotRestorePrepareResponse(new ArrayList<>(cacheCfgs.values()), partIds); } /** @@ -316,10 +323,10 @@ private void ensureCacheAbsent(String name) throws IllegalStateException { * Completes the verification phase and starts the restore performing phase if there were no errors. * * @param reqId Request ID. - * @param res Results. + * @param results Results. * @param errs Errors. */ - private void finishPrepare(UUID reqId, Map res, Map errs) { + private void finishPrepare(UUID reqId, Map results, Map errs) { RestoreSnapshotFuture fut0 = fut; if (fut0.isDone() || fut0.interrupted() || !reqId.equals(fut0.context().requestId())) @@ -333,89 +340,72 @@ private void finishPrepare(UUID reqId, Map SnapshotRestoreContext opCtx = fut0.context(); - Set missedGroups = new HashSet<>(opCtx.groups()); + // First node with snapshot data. + UUID firstSnapshotDataNode = null; try { - for (CacheGroupSnapshotDetails grpDetails : mergeNodeResults(res)) { - CacheConfiguration cfg = F.first(grpDetails.configs()).config(); + Map> grpPartIds = new HashMap<>(); + List ccfgs = null; - String grpName = cfg.getGroupName() == null ? cfg.getName() : cfg.getGroupName(); + for (Map.Entry entry : results.entrySet()) { + SnapshotRestorePrepareResponse res = entry.getValue(); - int reqParts = cfg.getAffinity().partitions(); - int availParts = grpDetails.parts().size(); + if (res == null) + continue; - if (reqParts != availParts) { - throw new IgniteCheckedException("Cannot restore snapshot, not all partitions available [" + - "required=" + reqParts + ", avail=" + availParts + ", group=" + grpName + ']'); + if (firstSnapshotDataNode == null) { + firstSnapshotDataNode = entry.getKey(); + ccfgs = res.configs(); + } + else if (res.configs().size() != res.configs().size()) { + throw new IllegalStateException("Count of cache configs in shared group mismatch [" + + "node1=" + firstSnapshotDataNode + ", cnt=" + ccfgs.size() + + ", node2=" + entry.getKey() + ", cnt=" + res.configs().size() + + ", snapshot=" + opCtx.snapshotName() + ']'); } - missedGroups.remove(grpName); - - for (StoredCacheData cacheData : grpDetails.configs()) - opCtx.addCacheData(cacheData); + for (Map.Entry> e : res.partIds().entrySet()) + grpPartIds.computeIfAbsent(e.getKey(), v -> new HashSet<>()).addAll(e.getValue()); } - if (!missedGroups.isEmpty()) { + if (!grpPartIds.keySet().containsAll(opCtx.groups())) { + Set missedGroups = new HashSet<>(opCtx.groups()); + + missedGroups.removeAll(grpPartIds.keySet()); + throw new IllegalArgumentException("Cache group(s) not found in snapshot [groups=" + F.concat(missedGroups, ", ") + ", snapshot=" + opCtx.snapshotName() + ']'); } - } - catch (Exception e) { - fut0.onDone(e); - - return; - } - - if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) { - UUID metaUpdateNode = F.first(F.viewReadOnly(res.entrySet(), Map.Entry::getKey, e -> e.getValue() != null)); - performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, metaUpdateNode)); - } - } - - /** - * @param res Results from multiple nodes. - * @return A collection that contains information about the snapshot cache group(s) on all nodes. - */ - private Collection mergeNodeResults(Map res) { - Map> globalDetails = new HashMap<>(); - - for (Map.Entry entry : res.entrySet()) { - UUID currNodeId = entry.getKey(); - SnapshotRestorePrepareResponse nodeResp = entry.getValue(); + for (StoredCacheData cacheData : ccfgs) { + CacheConfiguration ccfg = cacheData.config(); - if (nodeResp == null) - continue; + String grpName = ccfg.getGroupName() != null ? ccfg.getGroupName() : ccfg.getName(); - for (CacheGroupSnapshotDetails nodeDetails : nodeResp.groups()) { - CacheConfiguration cfg = F.first(nodeDetails.configs()).config(); + Set partIds = grpPartIds.get(grpName); - String grpName = cfg.getGroupName() == null ? cfg.getName() : cfg.getGroupName(); - - T2 clusterDetailsPair = globalDetails.get(grpName); - - if (clusterDetailsPair == null) { - globalDetails.put(grpName, new T2<>(currNodeId, nodeDetails)); - - continue; - } + int reqParts = ccfg.getAffinity().partitions(); + int availParts = partIds.size(); - CacheGroupSnapshotDetails clusterDetails = clusterDetailsPair.get2(); - - int currCfgCnt = nodeDetails.configs().size(); - int savedCfgCnt = clusterDetails.configs().size(); - - if (currCfgCnt != savedCfgCnt) { - throw new IllegalStateException("Count of cache configs in shared group mismatch [" + - "node1=" + clusterDetailsPair.get1() + ", cnt=" + savedCfgCnt + - ", node2=" + currNodeId + ", cnt=" + nodeDetails.configs().size() + ']'); + if (reqParts != availParts) { + throw new IgniteCheckedException("Cannot restore snapshot, not all partitions available [" + + "required=" + reqParts + + ", avail=" + availParts + + ", group=" + grpName + + ", snapshot=" + opCtx.snapshotName() + ']'); } - clusterDetails.parts().addAll(nodeDetails.parts()); + opCtx.addCacheData(cacheData); } } + catch (Exception e) { + fut0.onDone(e); + + return; + } - return F.viewReadOnly(globalDetails.values(), IgniteBiTuple::get2); + if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) + performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, firstSnapshotDataNode)); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java index 18e3b20f6109ef..e0b3ed1217ef9e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java @@ -19,6 +19,9 @@ import java.io.Serializable; import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.ignite.internal.processors.cache.StoredCacheData; /** * Snapshot restore prepare operation single node validation response. @@ -27,7 +30,9 @@ public class SnapshotRestorePrepareResponse implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; - private List groups; + private List ccfgs; + + private Map> partIds; /** */ public SnapshotRestorePrepareResponse() { @@ -37,12 +42,18 @@ public SnapshotRestorePrepareResponse() { /** * @param groups List of cache groups snapshot details. */ - public SnapshotRestorePrepareResponse(List groups) { - this.groups = groups; + public SnapshotRestorePrepareResponse(List ccfgs, Map> partIds) { + this.ccfgs = ccfgs; + this.partIds = partIds; } /** todo */ - public List groups() { - return groups; + public List configs() { + return ccfgs; + } + + public Map> partIds() { + return partIds; } + } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index ccd2ead469a784..f815b3eda35eb4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -657,6 +657,10 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { IgniteEx grid = grid(nodeIdx); + GridTestUtils.waitForCondition( + () -> !grid.context().cache().context().snapshotMgr().isCacheRestoring(null), + TIMEOUT); + File dir = resolveCacheDir(grid, cacheName); String errMsg = String.format("%s, dir=%s, exists=%b, files=%s", From 8b068e7cffb09460a461d4d21d701729a76b9a98 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 28 Jan 2021 18:32:31 +0300 Subject: [PATCH 21/98] IGNITE-13805 Meta update move to binary processor. --- .../CacheObjectBinaryProcessorImpl.java | 40 +++++++++++++ .../snapshot/IgniteSnapshotManager.java | 60 ------------------- .../SnapshotRestoreCacheGroupProcess.java | 19 +++--- .../snapshot/SnapshotRestoreContext.java | 30 +++++++--- .../SnapshotRestorePrepareRequest.java | 12 ++-- .../IgniteCacheObjectProcessor.java | 18 ++++++ 6 files changed, 92 insertions(+), 87 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 4d613d9aafc0b5..69d6acf6f638be 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -32,6 +33,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.BooleanSupplier; import javax.cache.CacheException; import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCheckedException; @@ -992,6 +994,44 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { } } + /** {@inheritDoc} */ + @Override public void checkMetadata(File metadataDir) throws IgniteCheckedException { + for (BinaryMetadata newMeta : readMetadata(metadataDir)) { + BinaryMetadata oldMeta = binaryMetadata(newMeta.typeId()); + + if (oldMeta != null) + BinaryUtils.mergeMetadata(oldMeta, newMeta, null); + } + } + + /** {@inheritDoc} */ + @Override public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException { + for (BinaryMetadata newMeta : readMetadata(metadataDir)) { + if (stopChecker.getAsBoolean()) + return; + + addMeta(newMeta.typeId(), newMeta.wrap(binaryContext()), false); + } + } + + /** + * @param metadataDir Directory containing binary metadata files. + * @return List of found metadata types. + * @throws IgniteCheckedException If failed. + */ + private Collection readMetadata(File metadataDir) throws IgniteCheckedException { + if (!metadataDir.exists()) + return Collections.emptyList(); + + ConcurrentMap metaCache = new ConcurrentHashMap<>(); + + BinaryMetadataFileStore binaryMetaFileStore = new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir); + + binaryMetaFileStore.restoreMetadata(); + + return F.viewReadOnly(metaCache.values(), BinaryMetadataHolder::metadata); + } + /** {@inheritDoc} */ @Override public BinaryObject buildEnum(String typeName, int ord) throws BinaryObjectException { A.notNullOrEmpty(typeName, "enum type name"); 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 dd6a9b1aeb7e5d..5159a3384cf1fb 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 @@ -20,7 +20,6 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.File; -import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; @@ -73,8 +72,6 @@ import org.apache.ignite.internal.IgniteFutureCancelledCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.NodeStoppingException; -import org.apache.ignite.internal.binary.BinaryMetadata; -import org.apache.ignite.internal.binary.BinaryUtils; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.events.DiscoveryCustomEvent; import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener; @@ -85,8 +82,6 @@ import org.apache.ignite.internal.processors.cache.CacheType; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; -import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware; import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow; @@ -135,7 +130,6 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; -import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.apache.ignite.thread.OomExceptionHandler; @@ -1117,60 +1111,6 @@ public List readSnapshotMetadatas(String snpName) { return restoreCacheGrpProcess.start(snpName, grpNames); } - /** - * @param snpName Snapshot name. - * @param checkCompatibility Don't update metadata, just check the compatibility of the snapshot metadata. - * @param failIfAbsent Throw an exception if the snapshot metadata folder doesn't exists. - * @param interruptClosure A closure to quickly interrupt the merge process. - * @throws IgniteCheckedException If failed. - */ - protected void mergeSnapshotMetadata( - String snpName, - boolean checkCompatibility, - boolean failIfAbsent, - BooleanSupplier interruptClosure - ) throws IgniteCheckedException { - File binDir = binaryWorkDir(snapshotLocalDir(snpName).getAbsolutePath(), pdsSettings.folderName()); - - if (!binDir.exists()) { - if (failIfAbsent) { - throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + - "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); - } - - return; - } - - Marshaller marshaller = cctx.kernalContext().config().getMarshaller(); - ClassLoader clsLdr = U.resolveClassLoader(cctx.kernalContext().config()); - CacheObjectBinaryProcessorImpl binProc = (CacheObjectBinaryProcessorImpl)cctx.kernalContext().cacheObjects(); - - for (File file : binDir.listFiles()) { - if (interruptClosure.getAsBoolean()) - return; - - try (FileInputStream in = new FileInputStream(file)) { - BinaryMetadata newMeta = U.unmarshal(marshaller, in, clsLdr); - - if (!checkCompatibility) { - binProc.addMeta(newMeta.typeId(), newMeta.wrap(binProc.binaryContext()), false); - - continue; - } - - BinaryMetadata oldMeta = binProc.binaryMetadata(newMeta.typeId()); - - if (oldMeta == null) - continue; - - BinaryUtils.mergeMetadata(oldMeta, newMeta, null); - } - catch (IOException e) { - throw new IgniteCheckedException("Failed to read metadata " + file, e); - } - } - } - /** * @param snpName Snapshot name. * @param grpName Cache group name. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index e61014a1d7ca22..bad4eb867c35e3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -51,6 +51,7 @@ import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; @@ -226,12 +227,9 @@ private IgniteInternalFuture prepare(SnapshotRes if (fut.isDone()) fut = new RestoreSnapshotFuture(); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + fut.init(new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.groups(), ctx)); - fut.init(new SnapshotRestoreContext( - req.requestId(), req.snapshotName(), req.requiredNodes(), req.groups(), snpMgr)); - - if (!snpMgr.snapshotLocalDir(req.snapshotName()).exists()) + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) return new GridFinishedFuture<>(); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -290,16 +288,13 @@ private IgniteInternalFuture prepare(SnapshotRes if (cacheCfgs.isEmpty()) return null; - Set cacheNames = new HashSet<>(req.groups()); - - cacheNames.addAll(cacheCfgs.keySet()); - - for (String cacheName : cacheNames) + for (String cacheName : F.concat(false, req.groups(), cacheCfgs.keySet())) ensureCacheAbsent(cacheName); - RestoreSnapshotFuture fut0 = fut; + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(req.snapshotName()).getAbsolutePath(), + ctx.pdsFolderResolver().resolveFolders().folderName()); - ctx.cache().context().snapshotMgr().mergeSnapshotMetadata(req.snapshotName(), true, false, fut0::interrupted); + ctx.cacheObjects().checkMetadata(binDir); return new SnapshotRestorePrepareResponse(new ArrayList<>(cacheCfgs.values()), partIds); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index bcba1ba5b4ae02..e508f1b23fa1f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -31,6 +31,7 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -39,6 +40,8 @@ import org.apache.ignite.internal.util.typedef.internal.S; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; + /** * Cache group restore from snapshot operation context. */ @@ -52,8 +55,8 @@ class SnapshotRestoreContext { /** List of baseline node IDs that must be alive to complete the operation. */ private final Set reqNodes; - /** Snapshot manager. */ - private final IgniteSnapshotManager snapshotMgr; + /** Kernal context. */ + private final GridKernalContext ctx; /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); @@ -69,17 +72,17 @@ class SnapshotRestoreContext { * @param snpName Snapshot name. * @param reqNodes List of baseline node IDs that must be alive to complete the operation. * @param grps List of cache group names to restore from the snapshot. - * @param snapshotMgr Snapshot manager. + * @param ctx Kernal context. */ public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Collection grps, - IgniteSnapshotManager snapshotMgr) { + GridKernalContext ctx) { for (String grpName : grps) this.grps.put(grpName, new GroupRestoreContext()); this.reqId = reqId; this.reqNodes = reqNodes; this.snpName = snpName; - this.snapshotMgr = snapshotMgr; + this.ctx = ctx; } /** @return Request ID. */ @@ -203,8 +206,17 @@ public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws if (stopChecker.getAsBoolean()) return; - if (updateMetadata) - snapshotMgr.mergeSnapshotMetadata(snpName, false, true, stopChecker); + if (updateMetadata) { + File binDir = binaryWorkDir(ctx.cache().context().snapshotMgr().snapshotLocalDir(snpName).getAbsolutePath(), + ctx.pdsFolderResolver().resolveFolders().folderName()); + + if (!binDir.exists()) { + throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + + "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); + } + + ctx.cacheObjects().updateMetadata(binDir, stopChecker); + } for (String grpName : groups()) { rollbackLock.lock(); @@ -212,7 +224,7 @@ public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws try { GroupRestoreContext grp = grps.get(grpName); - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, stopChecker, grp.files); + ctx.cache().context().snapshotMgr().restoreCacheGroupFiles(snpName, grpName, stopChecker, grp.files); } finally { rollbackLock.unlock(); @@ -234,7 +246,7 @@ public void rollback(String grpName) { if (grp == null || F.isEmpty(grp.files)) return; - snapshotMgr.rollbackRestoreOperation(grp.files); + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(grp.files); } finally { rollbackLock.unlock(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index 62a86ff9430e73..cdd75a6d86f3a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -44,18 +44,18 @@ public class SnapshotRestorePrepareRequest implements Serializable { /** List of baseline node IDs that must be alive to complete the operation. */ @GridToStringInclude - private final Set reqNodes; + private final Set nodes; /** * @param reqId Request ID. * @param snpName Snapshot name. * @param grps List of cache group names to restore from the snapshot. - * @param reqNodes List of baseline node IDs that must be alive to complete the operation. + * @param nodes List of baseline node IDs that must be alive to complete the operation. */ - public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection grps, Set reqNodes) { + public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection grps, Set nodes) { this.snpName = snpName; this.grps = grps; - this.reqNodes = reqNodes; + this.nodes = nodes; this.reqId = reqId; } @@ -83,8 +83,8 @@ public String snapshotName() { /** * @return List of baseline node IDs that must be alive to complete the operation. */ - public Set requiredNodes() { - return Collections.unmodifiableSet(reqNodes); + public Set nodes() { + return Collections.unmodifiableSet(nodes); } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java index 7ccfee02d7d4f1..37956bb42d5a0d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.util.Collection; import java.util.Map; +import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteBinary; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -306,6 +307,23 @@ public void updateMetadata(int typeId, String typeName, @Nullable String affKeyF */ public void saveMetadata(Collection types, File dir); + /** + * Check the compatibility of the binary metadata files stored in the specified directory. + * + * @param metadataDir Directory containing binary metadata files. + * @throws IgniteCheckedException If failed. + */ + public void checkMetadata(File metadataDir) throws IgniteCheckedException; + + /** + * Merge the binary metadata files stored in the specified directory. + * + * @param metadataDir Directory containing binary metadata files. + * @param stopChecker Prcoess interrupt checker. + * @throws IgniteCheckedException If failed. + */ + public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException; + /** * @param typeName Type name. * @param ord ordinal. From b519979fd8d187f5daee41145b2076a08a3b1f46 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 28 Jan 2021 21:12:23 +0300 Subject: [PATCH 22/98] IGNITE-13805 Minor code cleanp. --- .../snapshot/IgniteSnapshotManager.java | 28 ++++++++++--------- 1 file changed, 15 insertions(+), 13 deletions(-) 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 5159a3384cf1fb..871fdb4980c024 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 @@ -20,6 +20,7 @@ import java.io.BufferedInputStream; import java.io.BufferedOutputStream; import java.io.File; +import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; @@ -130,6 +131,8 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.resources.IgniteInstanceResource; import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.apache.ignite.thread.OomExceptionHandler; @@ -303,7 +306,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter private volatile ClusterSnapshotFuture lastSeenSnpFut = new ClusterSnapshotFuture(); /** Distributed process to restore cache group from the snapshot. */ - private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProcess; + private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProc; /** * @param ctx Kernal context. @@ -321,7 +324,7 @@ public IgniteSnapshotManager(GridKernalContext ctx) { marsh = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName()); - restoreCacheGrpProcess = new SnapshotRestoreCacheGroupProcess(ctx); + restoreCacheGrpProc = new SnapshotRestoreCacheGroupProcess(ctx); } /** @@ -418,7 +421,7 @@ public static String partDeltaFileName(int partId) { } } - restoreCacheGrpProcess.onNodeLeft(leftNodeId); + restoreCacheGrpProc.onNodeLeft(leftNodeId); } } finally { @@ -432,7 +435,7 @@ public static String partDeltaFileName(int partId) { busyLock.block(); try { - restoreCacheGrpProcess.stop("Node is stopping."); + restoreCacheGrpProc.stop("Node is stopping."); // Try stop all snapshot processing if not yet. for (SnapshotFutureTask sctx : locSnpTasks.values()) @@ -468,7 +471,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onDeActivate(GridKernalContext kctx) { - restoreCacheGrpProcess.stop("The cluster has been deactivated."); + restoreCacheGrpProc.stop("The cluster has been deactivated."); } /** @@ -773,7 +776,7 @@ public boolean isSnapshotCreating() { * @return {@code True} if the restore operation is in progress. */ public boolean isCacheRestoring(@Nullable String name) { - return restoreCacheGrpProcess.inProgress(name); + return restoreCacheGrpProc.inProgress(name); } /** @@ -784,7 +787,7 @@ public boolean isCacheRestoring(@Nullable String name) { * @param err Error if any. */ public void afterRestoredCacheStarted(String cacheName, @Nullable String grpName, @Nullable Throwable err) { - restoreCacheGrpProcess.handleCacheStart(cacheName, grpName, err); + restoreCacheGrpProc.handleCacheStart(cacheName, grpName, err); } /** @@ -1108,7 +1111,7 @@ public List readSnapshotMetadatas(String snpName) { /** {@inheritDoc} */ @Override public IgniteFuture restoreCacheGroups(String snpName, Collection grpNames) { - return restoreCacheGrpProcess.start(snpName, grpNames); + return restoreCacheGrpProc.start(snpName, grpNames); } /** @@ -1199,16 +1202,15 @@ protected void rollbackRestoreOperation(Collection files) { * @return Local path to the cache directory. */ public File resolveSnapshotCacheDir(String snpName, String cacheName) { - File workDir = snapshotLocalDir(snpName); + File dbDir = Paths.get(snapshotLocalDir(snpName).getAbsolutePath(), + DFLT_STORE_DIR, pdsSettings.folderName()).toFile(); - String dbPath = DFLT_STORE_DIR + File.separator + pdsSettings.folderName() + File.separator; - - File cacheDir = new File(workDir, dbPath + CACHE_DIR_PREFIX + cacheName); + File cacheDir = new File(dbDir, CACHE_DIR_PREFIX + cacheName); if (cacheDir.exists()) return cacheDir; - return new File(workDir, dbPath + CACHE_GRP_DIR_PREFIX + cacheName); + return new File(dbDir, CACHE_GRP_DIR_PREFIX + cacheName); } /** {@inheritDoc} */ From 1e0a3c9653623036fe77e43dbb0ed7182625f3fa Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 29 Jan 2021 11:32:08 +0300 Subject: [PATCH 23/98] IGNITE-13805 Extract context from future (wip). --- .../SnapshotRestoreCacheGroupProcess.java | 66 ++++++++----------- 1 file changed, 26 insertions(+), 40 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index bad4eb867c35e3..e4eaafb4c1e6f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -43,7 +43,6 @@ import org.apache.ignite.internal.util.future.GridFutureAdapter; import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.future.IgniteFutureImpl; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; 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.S; @@ -77,6 +76,9 @@ public class SnapshotRestoreCacheGroupProcess { /** The future to be completed when the cache restore process is complete. */ private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(); + /** Snapshot restore operation context. */ + private volatile SnapshotRestoreContext opCtx; + /** * @param ctx Kernal context. */ @@ -149,7 +151,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames public boolean inProgress(@Nullable String cacheName) { RestoreSnapshotFuture fut0 = fut; - return !staleFuture(fut0) && (cacheName == null || fut0.context().containsCache(cacheName)); + return !staleFuture(fut0) && (cacheName == null || opCtx.containsCache(cacheName)); } /** @@ -157,7 +159,7 @@ public boolean inProgress(@Nullable String cacheName) { * @return {@code True} if the future completed or not initiated. */ public boolean staleFuture(RestoreSnapshotFuture fut) { - return fut.isDone() || fut.context() == null; + return fut.isDone() || opCtx == null; } /** @@ -171,7 +173,7 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab if (staleFuture(fut0)) return; - fut0.context().processCacheStart(cacheName, grpName, err, ctx.getSystemExecutorService(), fut0); + opCtx.processCacheStart(cacheName, grpName, err, ctx.getSystemExecutorService(), fut0); } /** @@ -185,7 +187,7 @@ public void onNodeLeft(UUID leftNodeId) { if (staleFuture(fut0)) return; - if (fut0.context().nodes().contains(leftNodeId)) { + if (opCtx.nodes().contains(leftNodeId)) { fut0.onDone(new IgniteException(OP_REJECT_MSG + "Baseline node has left the cluster [nodeId=" + leftNodeId + ']')); } @@ -227,7 +229,9 @@ private IgniteInternalFuture prepare(SnapshotRes if (fut.isDone()) fut = new RestoreSnapshotFuture(); - fut.init(new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.groups(), ctx)); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.groups(), ctx); + + fut.listen(f -> opCtx = null); if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) return new GridFinishedFuture<>(); @@ -324,7 +328,7 @@ private void ensureCacheAbsent(String name) throws IllegalStateException { private void finishPrepare(UUID reqId, Map results, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.interrupted() || !reqId.equals(fut0.context().requestId())) + if (fut0.isDone() || fut0.interrupted() || !reqId.equals(opCtx.requestId())) return; if (!errs.isEmpty()) { @@ -333,7 +337,7 @@ private void finishPrepare(UUID reqId, Map return; } - SnapshotRestoreContext opCtx = fut0.context(); + SnapshotRestoreContext opCtx0 = opCtx; // First node with snapshot data. UUID firstSnapshotDataNode = null; @@ -356,20 +360,20 @@ else if (res.configs().size() != res.configs().size()) { throw new IllegalStateException("Count of cache configs in shared group mismatch [" + "node1=" + firstSnapshotDataNode + ", cnt=" + ccfgs.size() + ", node2=" + entry.getKey() + ", cnt=" + res.configs().size() + - ", snapshot=" + opCtx.snapshotName() + ']'); + ", snapshot=" + opCtx0.snapshotName() + ']'); } for (Map.Entry> e : res.partIds().entrySet()) grpPartIds.computeIfAbsent(e.getKey(), v -> new HashSet<>()).addAll(e.getValue()); } - if (!grpPartIds.keySet().containsAll(opCtx.groups())) { - Set missedGroups = new HashSet<>(opCtx.groups()); + if (!grpPartIds.keySet().containsAll(opCtx0.groups())) { + Set missedGroups = new HashSet<>(opCtx0.groups()); missedGroups.removeAll(grpPartIds.keySet()); throw new IllegalArgumentException("Cache group(s) not found in snapshot [groups=" + - F.concat(missedGroups, ", ") + ", snapshot=" + opCtx.snapshotName() + ']'); + F.concat(missedGroups, ", ") + ", snapshot=" + opCtx0.snapshotName() + ']'); } for (StoredCacheData cacheData : ccfgs) { @@ -387,10 +391,10 @@ else if (res.configs().size() != res.configs().size()) { "required=" + reqParts + ", avail=" + availParts + ", group=" + grpName + - ", snapshot=" + opCtx.snapshotName() + ']'); + ", snapshot=" + opCtx0.snapshotName() + ']'); } - opCtx.addCacheData(cacheData); + opCtx0.addCacheData(cacheData); } } catch (Exception e) { @@ -413,18 +417,18 @@ private IgniteInternalFuture perform(SnapshotRes if (fut0.isDone() || fut0.interrupted()) return new GridFinishedFuture<>(); - SnapshotRestoreContext opCtx = fut0.context(); + SnapshotRestoreContext opCtx0 = opCtx; - if (!req.requestId().equals(opCtx.requestId())) + if (!req.requestId().equals(opCtx0.requestId())) return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); GridFutureAdapter retFut = new GridFutureAdapter<>(); try { - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snapshotName()).exists()) + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx0.snapshotName()).exists()) return new GridFinishedFuture<>(); - for (StoredCacheData cfg : opCtx.configs()) { + for (StoredCacheData cfg : opCtx0.configs()) { if (!F.isEmpty(cfg.config().getGroupName())) ensureCacheAbsent(cfg.config().getName()); } @@ -433,7 +437,7 @@ private IgniteInternalFuture perform(SnapshotRes ctx.getSystemExecutorService().submit(() -> { try { - opCtx.restore(updateMeta, fut0::interrupted); + opCtx0.restore(updateMeta, fut0::interrupted); retFut.onDone(); } @@ -456,7 +460,7 @@ private IgniteInternalFuture perform(SnapshotRes private void finishPerform(UUID reqId, Map res, Map errs) { RestoreSnapshotFuture fut0 = fut; - if (fut0.isDone() || fut0.interrupted() || !reqId.equals(fut0.context().requestId())) + if (fut0.isDone() || fut0.interrupted() || !reqId.equals(opCtx.requestId())) return; Exception failure = F.first(errs.values()); @@ -470,7 +474,7 @@ private void finishPerform(UUID reqId, Map if (!U.isLocalNodeCoordinator(ctx.discovery())) return; - ctx.cache().dynamicStartCachesByStoredConf(fut0.context().configs(), true, true, false, null, true); + ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true); } /** */ @@ -478,24 +482,6 @@ private class RestoreSnapshotFuture extends GridFutureAdapter { /** The exception that led to the interruption of the process. */ private final AtomicReference errRef = new AtomicReference<>(); - /** Snapshot restore operation context. */ - @GridToStringInclude - private volatile SnapshotRestoreContext ctx; - - /** - * @return Cache group restore from snapshot operation context. - */ - public SnapshotRestoreContext context() { - return ctx; - } - - /** - * @param ctx Cache group restore from snapshot operation context. - */ - public void init(SnapshotRestoreContext ctx) { - this.ctx = ctx; - } - /** * @return Interrupted flag. */ @@ -509,7 +495,7 @@ public boolean interrupted() { return super.onDone(res, err, cancel); if (errRef.compareAndSet(null, err)) { - SnapshotRestoreContext opCtx0 = ctx; + SnapshotRestoreContext opCtx0 = opCtx; Set grpNames = opCtx0.groups(); From a5ad7707f974ed97b2a893a3c0bc3049e57fc76c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 12 Feb 2021 10:36:13 +0300 Subject: [PATCH 24/98] IGNITE-13805 Refactoring, prepare through compute (wip). --- .../processors/cache/CachesRegistry.java | 2 +- .../processors/cache/GridCacheProcessor.java | 7 +- .../GridCacheDatabaseSharedManager.java | 2 +- .../file/FilePageStoreManager.java | 20 - .../snapshot/IgniteSnapshotManager.java | 87 ++- .../SnapshotRestoreCacheGroupProcess.java | 496 +++++++++--------- .../snapshot/SnapshotRestoreContext.java | 139 ++--- .../snapshot/SnapshotRestoreRequest.java | 111 ++++ ...onse.java => SnapshotRestoreResponse.java} | 4 +- ...va => SnapshotRestoreRollbackRequest.java} | 23 +- .../SnapshotRestoreRollbackResponse.java | 45 ++ .../SnapshotRestoreVerificatioTask.java | 184 +++++++ ...va => SnapshotRestoreVerificationArg.java} | 40 +- ...=> SnapshotRestoreVerificationResult.java} | 37 +- .../util/distributed/DistributedProcess.java | 11 +- .../snapshot/AbstractSnapshotSelfTest.java | 3 + .../IgniteClusterSnapshoRestoreSelfTest.java | 271 +++++++--- 17 files changed, 923 insertions(+), 559 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestorePerformResponse.java => SnapshotRestoreResponse.java} (88%) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestorePerformRequest.java => SnapshotRestoreRollbackRequest.java} (69%) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestorePrepareRequest.java => SnapshotRestoreVerificationArg.java} (60%) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestorePrepareResponse.java => SnapshotRestoreVerificationResult.java} (63%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java index 6010cc1d627178..37d3b2b5bc3a80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java @@ -186,7 +186,7 @@ public IgniteInternalFuture update(ExchangeActions exchActions) { for (ExchangeActions.CacheGroupActionData stopAction : exchActions.cacheGroupsToStop()) { CacheGroupDescriptor rmvd = unregisterGroup(stopAction.descriptor().groupId()); - assert rmvd != null : stopAction.descriptor().cacheOrGroupName(); +// assert rmvd != null : stopAction.descriptor().cacheOrGroupName(); } for (ExchangeActions.CacheActionData req : exchActions.cacheStopRequests()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index a8c2e16be1726d..4b1b508716c226 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -2987,11 +2987,6 @@ public void completeCacheStartFuture(DynamicCacheChangeRequest req, boolean succ if (fut != null) fut.onDone(success, err); } - - if (req.restoredCache()) { - ctx.cache().context().snapshotMgr().afterRestoredCacheStarted(req.cacheName(), - req.startCacheConfiguration().getGroupName(), err); - } } /** @@ -5444,6 +5439,8 @@ private class CacheRecoveryLifecycle implements MetastorageLifecycleListener, Da /** {@inheritDoc} */ @Override public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException { + ctx.cache().context().snapshotMgr().cleanupRestoredCacheGroups(metastorage); + CacheJoinNodeDiscoveryData data = locCfgMgr.restoreCacheConfigurations(); cachesInfo.onStart(data); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index e6ca08c968ef01..bb6950102f4b46 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -1357,7 +1357,7 @@ else if (regCfg.getPageEvictionMode() != DataPageEvictionMode.DISABLED) { /** * @param partFile Partition file. */ - private int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException { + public int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException { FileIOFactory ioFactory = persistenceCfg.getFileIOFactory(); try (FileIO fileIO = ioFactory.create(partFile.toFile())) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java index 2611234cd636f3..24954a40888abf 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/file/FilePageStoreManager.java @@ -38,7 +38,6 @@ 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; @@ -1098,25 +1097,6 @@ private void readCacheGroupCaches(File grpDir, Map ccfg } } - /** - * @param cacheDIr Directory with partition files. - * @return Set of partition file IDs found in the directory. - */ - public Set scanPartitionIds(File cacheDIr) { - Set partIds = new HashSet<>(); - - for (String name : cacheDIr.list((dir, name) -> name.startsWith(PART_FILE_PREFIX))) { - if (new File(cacheDIr, name).isDirectory()) - continue; - - String partId = name.substring(PART_FILE_PREFIX.length(), name.indexOf('.')); - - partIds.add(Integer.parseInt(partId)); - } - - return partIds; - } - /** * @param conf File with stored cache data. * @return Cache data. 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 871fdb4980c024..a30f3520a7a9ef 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 @@ -219,6 +219,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Metastorage key to save currently running snapshot. */ public static final String SNP_RUNNING_KEY = "snapshot-running"; + /** Metastorage key to save currently restoring cache groups. */ + public static final String RESTORE_GRP_KEY = "snapshotRestoreGroups"; + /** Snapshot metrics prefix. */ public static final String SNAPSHOT_METRICS = "snapshot"; @@ -263,6 +266,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Marshaller. */ private final Marshaller marsh; + /** Distributed process to restore cache group from the snapshot. */ + private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProc; + /** Resolved persistent data storage settings. */ private volatile PdsFolderSettings pdsSettings; @@ -302,12 +308,12 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** {@code true} if recovery process occurred for snapshot. */ private volatile boolean recovered; + /** {@code true} if recovery process occurred for snapshot restore. */ + private volatile boolean restoreRecovered; + /** Last seen cluster snapshot operation. */ private volatile ClusterSnapshotFuture lastSeenSnpFut = new ClusterSnapshotFuture(); - /** Distributed process to restore cache group from the snapshot. */ - private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProc; - /** * @param ctx Kernal context. */ @@ -435,7 +441,7 @@ public static String partDeltaFileName(int partId) { busyLock.block(); try { - restoreCacheGrpProc.stop("Node is stopping."); + restoreCacheGrpProc.stop(new NodeStoppingException("Node is stopping.")); // Try stop all snapshot processing if not yet. for (SnapshotFutureTask sctx : locSnpTasks.values()) @@ -466,12 +472,22 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onActivate(GridKernalContext kctx) { - // No-op. + try { + if (metaStorage == null) + return; + + // Remove metastorage key used for recovery in case cluster has been + // deactivated and process was not able to do this. + updateRecoveryDataForRestoredGroups(null); + } + catch (IgniteCheckedException e) { + log.warning("Unable to remove key from metastorage.", e); + } } /** {@inheritDoc} */ @Override public void onDeActivate(GridKernalContext kctx) { - restoreCacheGrpProc.stop("The cluster has been deactivated."); + restoreCacheGrpProc.stop(new IgniteCheckedException("The cluster has been deactivated.")); } /** @@ -779,17 +795,6 @@ public boolean isCacheRestoring(@Nullable String name) { return restoreCacheGrpProc.inProgress(name); } - /** - * Callback from cache startup during cache group restore operation. - * - * @param cacheName Started cache name. - * @param grpName Started cache group name. - * @param err Error if any. - */ - public void afterRestoredCacheStarted(String cacheName, @Nullable String grpName, @Nullable Throwable err) { - restoreCacheGrpProc.handleCacheStart(cacheName, grpName, err); - } - /** * @return List of all known snapshots on the local node. */ @@ -1173,6 +1178,27 @@ protected void restoreCacheGroupFiles( } } + /** + * @param dirs Cache group directory names, required to clean up in case of node failure during the restore process. + * @throws IgniteCheckedException If failed. + */ + protected void updateRecoveryDataForRestoredGroups(@Nullable List dirs) throws IgniteCheckedException { + if (!cctx.kernalContext().state().clusterState().state().active()) + throw new IgniteCheckedException("Unable to update key in metastorage - cluster is not active."); + + cctx.database().checkpointReadLock(); + + try { + if (dirs == null) + metaStorage.remove(RESTORE_GRP_KEY); + else + metaStorage.write(RESTORE_GRP_KEY, new ArrayList<>(dirs)); + } + finally { + cctx.database().checkpointReadUnlock(); + } + } + /** * @param files Collection of files to delete. */ @@ -1223,6 +1249,11 @@ public File resolveSnapshotCacheDir(String snpName, String cacheName) { recovered = false; } + + if (restoreRecovered) + updateRecoveryDataForRestoredGroups(null); + + restoreRecovered = false; } /** {@inheritDoc} */ @@ -1246,6 +1277,28 @@ public File resolveSnapshotCacheDir(String snpName, String cacheName) { } } + /** + * Clean up restored cache group folders if a node failed during restore process. + * + * @param metaStorage Read-only meta storage. + * @throws IgniteCheckedException If failed. + */ + public void cleanupRestoredCacheGroups(ReadOnlyMetastorage metaStorage) throws IgniteCheckedException { + Collection grps = (Collection)metaStorage.read(RESTORE_GRP_KEY); + + if (grps == null) + return; + + for (String grpDirName : grps) { + File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), DFLT_STORE_DIR + + File.separator + pdsSettings.folderName() + File.separator + grpDirName, false); + + U.delete(cacheDir); + } + + restoreRecovered = true; + } + /** * @param evt Discovery event to check. * @return {@code true} if exchange started by snapshot operation. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index e4eaafb4c1e6f7..0f920e700f0008 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,26 +17,25 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.io.File; import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; +import java.util.Collections; import java.util.HashSet; -import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; -import org.apache.ignite.binary.BinaryObjectException; -import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -45,14 +44,13 @@ import org.apache.ignite.internal.util.future.IgniteFutureImpl; 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.S; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_FINISH; 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; /** * Distributed process to restore cache group from the snapshot. @@ -65,20 +63,26 @@ public class SnapshotRestoreCacheGroupProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess prepareRestoreProc; + private final DistributedProcess prepareRestoreProc; - /** Cache group restore perform phase. */ - private final DistributedProcess performRestoreProc; + /** Cache group restore cache start phase. */ + private final DistributedProcess cacheStartProc; + + /** Cache group restore rollback phase. */ + private final DistributedProcess rollbackRestoreProc; /** Logger. */ private final IgniteLogger log; /** The future to be completed when the cache restore process is complete. */ - private volatile RestoreSnapshotFuture fut = new RestoreSnapshotFuture(); + private volatile GridFutureAdapter fut = new GridFutureAdapter<>(); /** Snapshot restore operation context. */ private volatile SnapshotRestoreContext opCtx; + /** The exception that led to the interruption of the process. */ + private final AtomicReference errRef = new AtomicReference<>(); + /** * @param ctx Kernal context. */ @@ -87,10 +91,14 @@ public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { log = ctx.log(getClass()); - prepareRestoreProc = - new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, this::prepare, this::finishPrepare); - performRestoreProc = - new DistributedProcess<>(ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, this::perform, this::finishPerform); + prepareRestoreProc = new DistributedProcess<>( + ctx, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, this::prepare, this::finishPrepare); + + cacheStartProc = new DistributedProcess<>( + ctx, RESTORE_CACHE_GROUP_SNAPSHOT_START, this::cacheStart, this::finishCacheStart); + + rollbackRestoreProc = new DistributedProcess<>( + ctx, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, this::rollback, this::finishRollback); fut.onDone(); } @@ -117,7 +125,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames DiscoveryDataClusterState clusterState = ctx.state().clusterState(); - if (!clusterState.state().active()) + if (ctx.state().clusterState().state() != ClusterState.ACTIVE) return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); if (!clusterState.hasBaselineTopology()) { @@ -130,50 +138,87 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "A cluster snapshot operation is in progress.")); } - Set srvNodeIds = new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); + Set bltNodeIds = baselineNodes(); + + fut = new GridFutureAdapter<>(); + + ((ClusterGroupAdapter)ctx.cluster().get().forNodeIds(bltNodeIds)).compute().executeAsync( + new SnapshotRestoreVerificatioTask(), new SnapshotRestoreVerificationArg(snpName, cacheGrpNames)).listen( + f -> { + try { + SnapshotRestoreVerificationResult res = f.get(); + + Set foundGrps = res == null ? Collections.emptySet() : res.configs().stream() + .map(v -> v.config().getGroupName() != null ? v.config().getGroupName() : v.config().getName()) + .collect(Collectors.toSet()); + + if (!foundGrps.containsAll(cacheGrpNames)) { + Set missedGroups = new HashSet<>(cacheGrpNames); + + missedGroups.removeAll(foundGrps); + + fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + + "Cache group(s) was not found in the snapshot [groups=" + + F.concat(missedGroups, ", ") + ", snapshot=" + snpName + ']')); - SnapshotRestorePrepareRequest req = - new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, cacheGrpNames, srvNodeIds); + return; + } - fut = new RestoreSnapshotFuture(); + SnapshotRestoreRequest req = new SnapshotRestoreRequest( + UUID.randomUUID(), snpName, bltNodeIds, res.configs(), res.localNodeId()); - prepareRestoreProc.start(req.requestId(), req); + prepareRestoreProc.start(req.requestId(), req); + } catch (Throwable t) { + fut.onDone(new IgniteException(OP_REJECT_MSG + t.getMessage(), t)); + } + } + ); return new IgniteFutureImpl<>(fut); } + /** + * @return Set of current baseline node IDs. + */ + private Set baselineNodes() { + return new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); + } + /** * Check if the cache group restore process is currently running. * * @return {@code True} if cache group restore process is currently running. */ public boolean inProgress(@Nullable String cacheName) { - RestoreSnapshotFuture fut0 = fut; + IgniteInternalFuture fut0 = fut; return !staleFuture(fut0) && (cacheName == null || opCtx.containsCache(cacheName)); } /** - * @param fut The future of cache snapshot restore operation. - * @return {@code True} if the future completed or not initiated. + * @return Interrupted flag. */ - public boolean staleFuture(RestoreSnapshotFuture fut) { - return fut.isDone() || opCtx == null; + private boolean interrupted() { + return errRef.get() != null; } /** - * @param cacheName Started cache name. - * @param grpName Started cache group name. - * @param err Error if any. + * Interrupt process. + * + * @param err Error. + * @return {@code True} if process has been interrupted by this call. */ - public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullable Throwable err) { - RestoreSnapshotFuture fut0 = fut; - - if (staleFuture(fut0)) - return; + private boolean interrupt(Exception err) { + return errRef.compareAndSet(null, err); + } - opCtx.processCacheStart(cacheName, grpName, err, ctx.getSystemExecutorService(), fut0); + /** + * @param fut The future of cache snapshot restore operation. + * @return {@code True} if the future completed or not initiated. + */ + public boolean staleFuture(IgniteInternalFuture fut) { + return fut.isDone() || opCtx == null; } /** @@ -182,14 +227,14 @@ public void handleCacheStart(String cacheName, @Nullable String grpName, @Nullab * @param leftNodeId Left node ID. */ public void onNodeLeft(UUID leftNodeId) { - RestoreSnapshotFuture fut0 = fut; + IgniteInternalFuture fut0 = fut; if (staleFuture(fut0)) return; if (opCtx.nodes().contains(leftNodeId)) { - fut0.onDone(new IgniteException(OP_REJECT_MSG + - "Baseline node has left the cluster [nodeId=" + leftNodeId + ']')); + interrupt(new IgniteException(OP_REJECT_MSG + + "Baseline node(s) has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -198,23 +243,36 @@ public void onNodeLeft(UUID leftNodeId) { * * @param reason Interruption reason. */ - public void stop(String reason) { - RestoreSnapshotFuture fut0 = fut; + public void stop(Exception reason) { + IgniteInternalFuture fut0 = fut; if (staleFuture(fut0)) return; - fut0.onDone(new IgniteCheckedException("Restore process has been interrupted: " + reason)); + interrupt(reason); } /** - * Cache group napshot restore single node validation phase. + * Ensures that a cache with the specified name does not exist locally. * - * @param req Request to prepare snapshot restore. + * @param name Cache name. + * @throws IllegalStateException If cache with the specified name already exists. + */ + private void ensureCacheAbsent(String name) throws IllegalStateException { + int id = CU.cacheId(name); + + if (ctx.cache().cacheGroupDescriptors().containsKey(id) || ctx.cache().cacheDescriptor(id) != null) { + throw new IllegalStateException("Cache \"" + name + + "\" should be destroyed manually before perform restore operation."); + } + } + + /** + * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { - if (ctx.clientNode()) + private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { + if (!req.nodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); if (inProgress(null)) { @@ -222,234 +280,154 @@ private IgniteInternalFuture prepare(SnapshotRes new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); } - if (!ctx.state().clusterState().state().active()) + if (ctx.state().clusterState().state() != ClusterState.ACTIVE) return new GridFinishedFuture<>(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); // Skip creating future on initiator. if (fut.isDone()) - fut = new RestoreSnapshotFuture(); - - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.groups(), ctx); - - fut.listen(f -> opCtx = null); - - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(req.snapshotName()).exists()) - return new GridFinishedFuture<>(); - - GridFutureAdapter retFut = new GridFutureAdapter<>(); + fut = new GridFutureAdapter<>(); - ctx.getSystemExecutorService().submit(() -> { - try { - SnapshotRestorePrepareResponse res = prepare0(req); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), new HashSet<>(req.nodes()), req.configs(), ctx); - retFut.onDone(res); - } - catch (BinaryObjectException e) { - log.warning(OP_REJECT_MSG + "Incompatible binary types found", e); - - retFut.onDone(new IgniteException(OP_REJECT_MSG + "Incompatible binary types found: " + e.getMessage())); - } - catch (Throwable t) { - retFut.onDone(t); - } + fut.listen(f -> { + opCtx = null; + errRef.set(null); }); - return retFut; - } - - /** - * Reads locally stored cache configurations and verifies that the binary metadata can be merged from the snapshot. - * - * @param req Request to prepare snapshot restore. - * @return Response to prepare snapshot restore. - * @throws IgniteCheckedException If failed. - */ - private @Nullable SnapshotRestorePrepareResponse prepare0( - SnapshotRestorePrepareRequest req - ) throws IgniteCheckedException { - if (log.isInfoEnabled()) - log.info("Preparing to restore cache groups [groups=" + F.concat(req.groups(), ", ") + ']'); - - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + if (!baselineNodes().containsAll(req.nodes())) { + return new GridFinishedFuture<>( + new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster.")); + } - Map cacheCfgs = new HashMap<>(); - Map> partIds = new HashMap<>(); + SnapshotRestoreContext opCtx0 = opCtx; - // Collect cache configuration(s). - for (String grpName : req.groups()) { - File cacheDir = snapshotMgr.resolveSnapshotCacheDir(req.snapshotName(), grpName); + GridFutureAdapter retFut = new GridFutureAdapter<>(); - if (!cacheDir.exists()) - return null; + try { + for (String grpName : opCtx0.groups()) + ensureCacheAbsent(grpName); - FilePageStoreManager pageStoreMgr = (FilePageStoreManager)ctx.cache().context().pageStore(); + for (StoredCacheData cfg : opCtx0.configs()) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - pageStoreMgr.readCacheConfigurations(cacheDir, cacheCfgs); + fut.listen(f -> { + try { + ctx.cache().context().snapshotMgr().updateRecoveryDataForRestoredGroups(null); + } + catch (IgniteCheckedException e) { + log.warning("Unable to reset restored groups.", e); + } + }); - partIds.put(grpName, pageStoreMgr.scanPartitionIds(cacheDir)); - } + ctx.cache().context().snapshotMgr().updateRecoveryDataForRestoredGroups(new ArrayList<>(opCtx0.groupDirs())); - if (cacheCfgs.isEmpty()) - return null; + if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx0.snapshotName()).exists()) + return new GridFinishedFuture<>(); - for (String cacheName : F.concat(false, req.groups(), cacheCfgs.keySet())) - ensureCacheAbsent(cacheName); + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); - File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(req.snapshotName()).getAbsolutePath(), - ctx.pdsFolderResolver().resolveFolders().folderName()); + ctx.getSystemExecutorService().submit(() -> { + try { + opCtx0.restore(updateMeta, this::interrupted); - ctx.cacheObjects().checkMetadata(binDir); + if (interrupted()) { + log.error("Snapshot restore process has been interrupted " + + "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', errRef.get()); - return new SnapshotRestorePrepareResponse(new ArrayList<>(cacheCfgs.values()), partIds); - } + opCtx0.rollback(); - /** - * Ensures that a cache with the specified name does not exist locally. - * - * @param name Cache name. - * @throws IllegalStateException If cache with the specified name already exists. - */ - private void ensureCacheAbsent(String name) throws IllegalStateException { - int id = CU.cacheId(name); + retFut.onDone(errRef.get()); + } + else + retFut.onDone(); + } + catch (Throwable t) { + retFut.onDone(t); + } + }); - if (ctx.cache().cacheDescriptor(id) != null || ctx.cache().cacheGroupDescriptor(id) != null) { - throw new IllegalStateException("Cache \"" + name + - "\" should be destroyed manually before perform restore operation."); + return retFut; + } catch (Exception e) { + return new GridFinishedFuture<>(e); } } /** - * Completes the verification phase and starts the restore performing phase if there were no errors. - * * @param reqId Request ID. - * @param results Results. + * @param res Results. * @param errs Errors. */ - private void finishPrepare(UUID reqId, Map results, Map errs) { - RestoreSnapshotFuture fut0 = fut; - - if (fut0.isDone() || fut0.interrupted() || !reqId.equals(opCtx.requestId())) - return; - - if (!errs.isEmpty()) { - fut0.onDone(F.firstValue(errs)); + private void finishPrepare(UUID reqId, Map res, Map errs) { + GridFutureAdapter fut0 = fut; + if (fut0.isDone() || !reqId.equals(opCtx.requestId())) return; - } - - SnapshotRestoreContext opCtx0 = opCtx; - - // First node with snapshot data. - UUID firstSnapshotDataNode = null; - - try { - Map> grpPartIds = new HashMap<>(); - List ccfgs = null; - - for (Map.Entry entry : results.entrySet()) { - SnapshotRestorePrepareResponse res = entry.getValue(); - - if (res == null) - continue; - - if (firstSnapshotDataNode == null) { - firstSnapshotDataNode = entry.getKey(); - ccfgs = res.configs(); - } - else if (res.configs().size() != res.configs().size()) { - throw new IllegalStateException("Count of cache configs in shared group mismatch [" + - "node1=" + firstSnapshotDataNode + ", cnt=" + ccfgs.size() + - ", node2=" + entry.getKey() + ", cnt=" + res.configs().size() + - ", snapshot=" + opCtx0.snapshotName() + ']'); - } - - for (Map.Entry> e : res.partIds().entrySet()) - grpPartIds.computeIfAbsent(e.getKey(), v -> new HashSet<>()).addAll(e.getValue()); - } - - if (!grpPartIds.keySet().containsAll(opCtx0.groups())) { - Set missedGroups = new HashSet<>(opCtx0.groups()); - missedGroups.removeAll(grpPartIds.keySet()); - - throw new IllegalArgumentException("Cache group(s) not found in snapshot [groups=" + - F.concat(missedGroups, ", ") + ", snapshot=" + opCtx0.snapshotName() + ']'); - } - - for (StoredCacheData cacheData : ccfgs) { - CacheConfiguration ccfg = cacheData.config(); - - String grpName = ccfg.getGroupName() != null ? ccfg.getGroupName() : ccfg.getName(); - - Set partIds = grpPartIds.get(grpName); - - int reqParts = ccfg.getAffinity().partitions(); - int availParts = partIds.size(); + Exception failure = F.first(errs.values()); - if (reqParts != availParts) { - throw new IgniteCheckedException("Cannot restore snapshot, not all partitions available [" + - "required=" + reqParts + - ", avail=" + availParts + - ", group=" + grpName + - ", snapshot=" + opCtx0.snapshotName() + ']'); - } + if (failure != null) { + opCtx.rollback(); - opCtx0.addCacheData(cacheData); - } - } - catch (Exception e) { - fut0.onDone(e); + fut0.onDone(failure); return; } - if (U.isLocalNodeCoordinator(ctx.discovery()) && !fut0.isDone()) - performRestoreProc.start(reqId, new SnapshotRestorePerformRequest(reqId, firstSnapshotDataNode)); + if (U.isLocalNodeCoordinator(ctx.discovery())) + cacheStartProc.start(reqId, new SnapshotRestoreRequest(reqId, null, null, null, null)); } /** - * @param req Request to perform snapshot restore. + * @param req Request to start restored cache groups. * @return Result future. */ - private IgniteInternalFuture perform(SnapshotRestorePerformRequest req) { - RestoreSnapshotFuture fut0 = fut; + private IgniteInternalFuture cacheStart(SnapshotRestoreRequest req) { + SnapshotRestoreContext opCtx0 = opCtx; - if (fut0.isDone() || fut0.interrupted()) + if (staleFuture(fut) || !req.requestId().equals(opCtx0.requestId())) return new GridFinishedFuture<>(); - SnapshotRestoreContext opCtx0 = opCtx; + if (!opCtx0.nodes().contains(ctx.localNodeId())) + return new GridFinishedFuture<>(); if (!req.requestId().equals(opCtx0.requestId())) return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); - GridFutureAdapter retFut = new GridFutureAdapter<>(); + if (ctx.state().clusterState().state() != ClusterState.ACTIVE) + return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Cluster state has been changed.")); - try { - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx0.snapshotName()).exists()) - return new GridFinishedFuture<>(); + if (!U.isLocalNodeCoordinator(ctx.discovery())) + return new GridFinishedFuture<>(); - for (StoredCacheData cfg : opCtx0.configs()) { - if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); - } + if (interrupted()) + return new GridFinishedFuture<>(errRef.get()); - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + if (!baselineNodes().containsAll(opCtx0.nodes())) + return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster.")); - ctx.getSystemExecutorService().submit(() -> { - try { - opCtx0.restore(updateMeta, fut0::interrupted); + GridFutureAdapter retFut = new GridFutureAdapter<>(); - retFut.onDone(); - } - catch (Throwable t) { - retFut.onDone(t); + if (log.isInfoEnabled()) { + log.info("Starting restored caches " + + "[snapshot=" + opCtx0.snapshotName() + + ", caches=" + F.viewReadOnly(opCtx0.configs(), c -> c.config().getName()) + ']'); + } + + ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true).listen( + f -> { + if (f.error() != null) { + log.error("Unable to start restored caches.", f.error()); + + retFut.onDone(f.error()); } - }); + else + retFut.onDone(); + } + ); - return retFut; - } catch (Exception e) { - return new GridFinishedFuture<>(e); - } + return retFut; } /** @@ -457,63 +435,67 @@ private IgniteInternalFuture perform(SnapshotRes * @param res Results. * @param errs Errors. */ - private void finishPerform(UUID reqId, Map res, Map errs) { - RestoreSnapshotFuture fut0 = fut; + private void finishCacheStart(UUID reqId, Map res, Map errs) { + GridFutureAdapter fut0 = fut; + SnapshotRestoreContext opCtx0 = opCtx; - if (fut0.isDone() || fut0.interrupted() || !reqId.equals(opCtx.requestId())) + if (staleFuture(fut0) || !reqId.equals(opCtx0.requestId())) return; Exception failure = F.first(errs.values()); - if (failure != null) { - fut0.onDone(failure); + if (failure == null && !res.keySet().containsAll(opCtx0.nodes())) { + Set leftNodes = new HashSet<>(opCtx0.nodes()); - return; + leftNodes.removeAll(res.keySet()); + + failure = new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster [nodeId=" + leftNodes + ']'); } - if (!U.isLocalNodeCoordinator(ctx.discovery())) + if (failure != null) { + if (U.isLocalNodeCoordinator(ctx.discovery())) + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); + return; + } - ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true); + fut0.onDone(); } - /** */ - private class RestoreSnapshotFuture extends GridFutureAdapter { - /** The exception that led to the interruption of the process. */ - private final AtomicReference errRef = new AtomicReference<>(); + /** + * @param req Request to rollback cache group restore process. + * @return Result future. + */ + private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { + if (staleFuture(fut) || !req.requestId().equals(opCtx.requestId())) + return new GridFinishedFuture<>(); - /** - * @return Interrupted flag. - */ - public boolean interrupted() { - return errRef.get() != null; - } + SnapshotRestoreContext opCtx0 = opCtx; - /** {@inheritDoc} */ - @Override protected boolean onDone(@Nullable Void res, @Nullable Throwable err, boolean cancel) { - if (err == null) - return super.onDone(res, err, cancel); + if (!opCtx0.nodes().contains(ctx.localNodeId())) + return new GridFinishedFuture<>(); - if (errRef.compareAndSet(null, err)) { - SnapshotRestoreContext opCtx0 = opCtx; + if (log.isInfoEnabled()) + log.info("Performing rollback routine for restored cache groups [groups=" + opCtx0.groups() + ']'); - Set grpNames = opCtx0.groups(); + opCtx0.rollback(); - log.error("Snapshot restore process has been interrupted " + - "[groups=" + grpNames + ", snapshot=" + opCtx0.snapshotName() + ']', err); + return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse(req.error())); + } - for (String grpName : grpNames) - opCtx0.rollback(grpName); + /** + * @param reqId Request ID. + * @param res Results. + * @param errs Errors. + */ + private void finishRollback(UUID reqId, Map res, Map errs) { + GridFutureAdapter fut0 = fut; - return super.onDone(res, err, cancel); - } + if (staleFuture(fut0) || !reqId.equals(opCtx.requestId())) + return; - return false; - } + SnapshotRestoreRollbackResponse resp = F.first(F.viewReadOnly(res.values(), v -> v, Objects::nonNull)); - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(RestoreSnapshotFuture.class, this); - } + fut0.onDone(resp.error()); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index e508f1b23fa1f3..af711d100dceea 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -27,20 +27,18 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.util.GridConcurrentHashSet; -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.S; -import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; /** * Cache group restore from snapshot operation context. @@ -71,13 +69,21 @@ class SnapshotRestoreContext { * @param reqId Request ID. * @param snpName Snapshot name. * @param reqNodes List of baseline node IDs that must be alive to complete the operation. - * @param grps List of cache group names to restore from the snapshot. + * @param configs Stored cache configurations. * @param ctx Kernal context. */ - public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Collection grps, + public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List configs, GridKernalContext ctx) { - for (String grpName : grps) - this.grps.put(grpName, new GroupRestoreContext()); + for (StoredCacheData cacheData : configs) { + String cacheName = cacheData.config().getName(); + + cacheCfgs.put(CU.cacheId(cacheName), cacheData); + + boolean shared = cacheData.config().getGroupName() != null; + + grps.computeIfAbsent( + shared ? cacheData.config().getGroupName() : cacheName, v -> new GroupRestoreContext(shared)); + } this.reqId = reqId; this.reqNodes = reqNodes; @@ -100,11 +106,21 @@ public String snapshotName() { return snpName; } - /** @return List of cache group names to restore from the snapshot. */ + /** + * @return List of cache group names to restore from the snapshot. + */ public Set groups() { return grps.keySet(); } + /** + * @return Names of the directories of the restored caches. + */ + public Collection groupDirs() { + return F.viewReadOnly(grps.entrySet(), + e -> (e.getValue().shared ? CACHE_GRP_DIR_PREFIX : CACHE_DIR_PREFIX) + e.getKey()); + } + /** * @param name Cache name. * @return {@code True} if the cache with the specified name is currently being restored. @@ -118,83 +134,6 @@ public Collection configs() { return cacheCfgs.values(); } - /** - * @param cacheData Stored cache data. - */ - public void addCacheData(StoredCacheData cacheData) { - String cacheName = cacheData.config().getName(); - - cacheCfgs.put(CU.cacheId(cacheName), cacheData); - - String grpName = cacheData.config().getGroupName(); - - if (grpName == null) - return; - - GroupRestoreContext grpCtx = grps.get(grpName); - - assert grpCtx != null : grpName; - - grpCtx.caches.add(cacheName); - } - - /** - * @param cacheName Cache name. - * @param grpName Group name. - * @param err Exception (if any). - * @param svc Executor service for asynchronous rollback. - * @param finishFut A future to be completed when all restored cache groups are started or rolled back. - */ - public void processCacheStart( - String cacheName, - @Nullable String grpName, - @Nullable Throwable err, - ExecutorService svc, - GridFutureAdapter finishFut - ) { - String grpName0 = grpName != null ? grpName : cacheName; - - GroupRestoreContext grp = grps.get(grpName0); - - // If any of shared caches has been started - we cannot rollback changes. - if (grp.caches.remove(cacheName) && err == null) - grp.started = true; - - if (!grp.caches.isEmpty()) { - if (err != null) - grp.startErr = err; - - return; - } - - if (err != null && !grp.started) { - svc.submit(() -> { - rollbackLock.lock(); - - try { - rollback(grpName0); - - if (grps.isEmpty()) - finishFut.onDone(err); - } - finally { - rollbackLock.unlock(); - } - }); - - return; - } - - rollbackLock.lock(); - - try { - if (grps.remove(grpName0) != null && grps.isEmpty()) - finishFut.onDone(null, err == null ? grp.startErr : err); - } finally { - rollbackLock.unlock(); - } - } - /** * Restore specified cache groups from the local snapshot directory. * @@ -234,20 +173,21 @@ public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws /** * Rollback changes made by process in specified cache group. - * - * @param grpName Cache group name. */ - public void rollback(String grpName) { + public void rollback() { rollbackLock.lock(); try { - GroupRestoreContext grp = grps.remove(grpName); + List grpNames = new ArrayList<>(groups()); - if (grp == null || F.isEmpty(grp.files)) - return; + for (String grpName : grpNames) { + GroupRestoreContext grp = grps.remove(grpName); - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(grp.files); - } finally { + if (grp != null) + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(grp.files); + } + } + finally { rollbackLock.unlock(); } } @@ -259,16 +199,13 @@ public void rollback(String grpName) { /** */ private static class GroupRestoreContext { - /** List of caches of the cache group. */ - final Set caches = new GridConcurrentHashSet<>(); - /** Files created in the cache group folder during a restore operation. */ final List files = new ArrayList<>(); - /** The flag indicates that one of the caches in this cache group has been started. */ - volatile boolean started; + final boolean shared; - /** An exception that was thrown when starting a shared cache group (if any). */ - volatile Throwable startErr; + private GroupRestoreContext(boolean shared) { + this.shared = shared; + } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java new file mode 100644 index 00000000000000..e4471b44fa6ce0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java @@ -0,0 +1,111 @@ +/* + * 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.List; +import java.util.UUID; +import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Request to prepare cache group restore from the snapshot. + */ +public class SnapshotRestoreRequest implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Request ID. */ + private final UUID reqId; + + /** Snapshot name. */ + private final String snpName; + + /** Baseline node IDs that must be alive to complete the operation. */ + private final Collection nodes; + + /** Stored cache configurations. */ + @GridToStringExclude + private final List ccfgs; + + /** Node ID from which to update the binary metadata. */ + private final UUID updateMetaNodeId; + + /** + * @param reqId Request ID. + * @param snpName Snapshot name. + * @param nodes Baseline node IDs that must be alive to complete the operation. + * @param ccfgs Stored cache configurations. + * @param updateMetaNodeId Node ID from which to update the binary metadata. + */ + public SnapshotRestoreRequest( + UUID reqId, + String snpName, + Collection nodes, + List ccfgs, + UUID updateMetaNodeId + ) { + this.reqId = reqId; + this.snpName = snpName; + this.nodes = nodes; + this.ccfgs = ccfgs; + this.updateMetaNodeId = updateMetaNodeId; + } + + /** + * @return Request ID. + */ + public UUID requestId() { + return reqId; + } + + /** + * @return Snapshot name. + */ + public String snapshotName() { + return snpName; + } + + /** + * @return Stored cache configurations. + */ + public List configs() { + return ccfgs; + } + + /** + * @return Baseline node IDs that must be alive to complete the operation. + */ + public Collection nodes() { + return nodes; + } + + /** + * @return Node ID from which to update the binary metadata. + */ + public UUID updateMetaNodeId() { + return updateMetaNodeId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestoreRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java similarity index 88% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java index df887d9fbfda6b..79601b00f5b581 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java @@ -20,9 +20,9 @@ import java.io.Serializable; /** - * Snapshot restore perform operation single node response. + * Snapshot restore operation single node response. */ -public class SnapshotRestorePerformResponse implements Serializable { +public class SnapshotRestoreResponse implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java similarity index 69% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java index a3df9bc9b46246..a5089aa09bae05 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePerformRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java @@ -20,27 +20,28 @@ import java.io.Serializable; import java.util.UUID; import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; /** - * Request to perform snapshot restore. + * Request to complete/rollback cache group restore process. */ -public class SnapshotRestorePerformRequest implements Serializable { +public class SnapshotRestoreRollbackRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; /** Request ID. */ private final UUID reqId; - /** Node ID from which to update the binary metadata. */ - private final UUID updateMetaNodeId; + /** Process execution error. */ + private final Throwable err; /** * @param reqId Request ID. - * @param updateMetaNodeId Node ID from which to update the binary metadata. + * @param err Process execution error. */ - public SnapshotRestorePerformRequest(UUID reqId, UUID updateMetaNodeId) { + public SnapshotRestoreRollbackRequest(UUID reqId, @Nullable Throwable err) { this.reqId = reqId; - this.updateMetaNodeId = updateMetaNodeId; + this.err = err; } /** @@ -51,14 +52,14 @@ public UUID requestId() { } /** - * @return Node ID from which to update the binary metadata. + * @return Process execution error. */ - public UUID updateMetaNodeId() { - return updateMetaNodeId; + public @Nullable Throwable error() { + return err; } /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SnapshotRestorePerformRequest.class, this); + return S.toString(SnapshotRestoreRollbackRequest.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java new file mode 100644 index 00000000000000..b0e609e07be310 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java @@ -0,0 +1,45 @@ +/* + * 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; + +/** + * Response of the final phase of cache group restore. + */ +public class SnapshotRestoreRollbackResponse implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Process execution error. */ + private final Throwable err; + + /** + * @param err Process execution error. + */ + public SnapshotRestoreRollbackResponse(Throwable err) { + this.err = err; + } + + /** + * @return Process execution error. + */ + public Throwable error() { + return err; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java new file mode 100644 index 00000000000000..5822ef1819772b --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java @@ -0,0 +1,184 @@ +/* + * 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.io.IOException; +import java.util.ArrayList; +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.binary.BinaryObjectException; +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.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; + +/** + * Verification task for restoring a cache group from a snapshot. + */ +public class SnapshotRestoreVerificatioTask extends + ComputeTaskAdapter { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** {@inheritDoc} */ + @Override public @NotNull Map map(List subgrid, + SnapshotRestoreVerificationArg arg) throws IgniteException { + Map jobs = new HashMap<>(); + + for (ClusterNode node : subgrid) + jobs.put(new SnapshotRestoreVerificationJob(arg), node); + + return jobs; + } + + /** {@inheritDoc} */ + @Override public SnapshotRestoreVerificationResult reduce(List results) throws IgniteException { + SnapshotRestoreVerificationResult firstRes = null; + + for (ComputeJobResult jobRes : results) { + SnapshotRestoreVerificationResult res = jobRes.getData(); + + if (res == null) + continue; + + if (firstRes == null) { + firstRes = res; + + continue; + } + + if (firstRes.configs().size() != res.configs().size()) { + throw new IgniteException("Count of cache configs mismatch [" + + "node1=" + firstRes.localNodeId() + ", cnt1=" + firstRes.configs().size() + + ", node2=" + res.localNodeId() + ", cnt2=" + res.configs().size() + ']'); + } + } + + return firstRes; + } + + /** {@inheritDoc} */ + @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) { + IgniteException e = res.getException(); + + // Don't failover this job, if topology changed - user should restart operation. + if (e != null) + throw e; + + return super.result(res, rcvd); + } + + /** */ + private static class SnapshotRestoreVerificationJob extends ComputeJobAdapter { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Auto-injected grid instance. */ + @IgniteInstanceResource + private transient IgniteEx ignite; + + /** Job argument. */ + private final SnapshotRestoreVerificationArg arg; + + /** + * @param arg Job argument. + */ + public SnapshotRestoreVerificationJob(SnapshotRestoreVerificationArg arg) { + this.arg = arg; + } + + /** {@inheritDoc} */ + @Override public Object execute() throws IgniteException { + assert !ignite.context().clientNode(); + + try { + return resolveRestoredConfigs(); + } + catch (BinaryObjectException e) { + throw new IgniteException("Incompatible binary types found: " + e.getMessage()); + } catch (IOException | IgniteCheckedException e) { + throw F.wrap(e); + } + } + + /** + * Collect cache configurations and verify binary compatibility of specified cache groups. + * + * @return List of stored cache configurations with local node ID. + * @throws IgniteCheckedException If the snapshot is incompatible. + * @throws IOException In case of I/O errors while reading the memory page size + */ + private SnapshotRestoreVerificationResult resolveRestoredConfigs() throws IgniteCheckedException, IOException { + GridCacheSharedContext cctx = ignite.context().cache().context(); + Map cacheCfgs = new HashMap<>(); + + // Collect cache configuration(s) and verify cache groups page size. + for (String grpName : arg.groups()) { + File cacheDir = cctx.snapshotMgr().resolveSnapshotCacheDir(arg.snapshotName(), grpName); + + if (!cacheDir.exists()) + return null; + + ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cacheCfgs); + + File[] parts = cacheDir.listFiles(f -> f.getName().startsWith(PART_FILE_PREFIX) && !f.isDirectory()); + + if (F.isEmpty(parts)) + continue; + + int pageSize = + ((GridCacheDatabaseSharedManager)cctx.database()).resolvePageSizeFromPartitionFile(parts[0].toPath()); + + if (pageSize != cctx.database().pageSize()) { + throw new IgniteCheckedException("Incompatible memory page size " + + "[snapshotPageSize=" + pageSize + + ", nodePageSize=" + cctx.database().pageSize() + + ", group=" + grpName + + ", snapshot=" + arg.snapshotName() + ']'); + } + } + + if (cacheCfgs.isEmpty()) + return null; + + File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(arg.snapshotName()).getAbsolutePath(), + ignite.context().pdsFolderResolver().resolveFolders().folderName()); + + ignite.context().cacheObjects().checkMetadata(binDir); + + return new SnapshotRestoreVerificationResult(new ArrayList<>(cacheCfgs.values()), ignite.localNode().id()); + } + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java similarity index 60% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java index cdd75a6d86f3a4..e7c2f01ee29856 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java @@ -19,22 +19,15 @@ import java.io.Serializable; import java.util.Collection; -import java.util.Collections; -import java.util.Set; -import java.util.UUID; import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.internal.S; /** - * Request to prepare snapshot restore. + * Verification task argument. */ -public class SnapshotRestorePrepareRequest implements Serializable { +public class SnapshotRestoreVerificationArg implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; - /** Request ID. */ - private final UUID reqId; - /** Snapshot name. */ private final String snpName; @@ -42,28 +35,13 @@ public class SnapshotRestorePrepareRequest implements Serializable { @GridToStringInclude private final Collection grps; - /** List of baseline node IDs that must be alive to complete the operation. */ - @GridToStringInclude - private final Set nodes; - /** - * @param reqId Request ID. * @param snpName Snapshot name. * @param grps List of cache group names to restore from the snapshot. - * @param nodes List of baseline node IDs that must be alive to complete the operation. */ - public SnapshotRestorePrepareRequest(UUID reqId, String snpName, Collection grps, Set nodes) { + public SnapshotRestoreVerificationArg(String snpName, Collection grps) { this.snpName = snpName; this.grps = grps; - this.nodes = nodes; - this.reqId = reqId; - } - - /** - * @return Request ID. - */ - public UUID requestId() { - return reqId; } /** @@ -79,16 +57,4 @@ public Collection groups() { public String snapshotName() { return snpName; } - - /** - * @return List of baseline node IDs that must be alive to complete the operation. - */ - public Set nodes() { - return Collections.unmodifiableSet(nodes); - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotRestorePrepareRequest.class, this); - } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java similarity index 63% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java index e0b3ed1217ef9e..dbb8d75fd00407 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java @@ -19,41 +19,42 @@ import java.io.Serializable; import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.UUID; import org.apache.ignite.internal.processors.cache.StoredCacheData; /** - * Snapshot restore prepare operation single node validation response. + * Result of a cache group restore verification job. */ -public class SnapshotRestorePrepareResponse implements Serializable { +public class SnapshotRestoreVerificationResult implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; - private List ccfgs; + /** List of stored cache configurations. */ + private final List ccfgs; - private Map> partIds; - - /** */ - public SnapshotRestorePrepareResponse() { - // No-op. - } + /** Local node ID. */ + private final UUID locNodeId; /** - * @param groups List of cache groups snapshot details. + * @param ccfgs List of stored cache configurations. + * @param locNodeId Local node ID. */ - public SnapshotRestorePrepareResponse(List ccfgs, Map> partIds) { + public SnapshotRestoreVerificationResult(List ccfgs, UUID locNodeId) { this.ccfgs = ccfgs; - this.partIds = partIds; + this.locNodeId = locNodeId; } - /** todo */ + /** + * @return List of stored cache configurations. + */ public List configs() { return ccfgs; } - public Map> partIds() { - return partIds; + /** + * @return Local node ID. + */ + public UUID localNodeId() { + return locNodeId; } - } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index ed3eab5a6e061b..67280ef6b9f902 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -448,13 +448,18 @@ public enum DistributedProcessType { PERFORMANCE_STATISTICS_ROTATE, /** - * Cache group restore from snapshot prepare process. + * Cache group restore prepare phase. */ RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, /** - * Cache group restore from snapshot perform process. + * Cache group restore cache start phase. */ - RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM + RESTORE_CACHE_GROUP_SNAPSHOT_START, + + /** + * Cache group restore rollback phase. + */ + RESTORE_CACHE_GROUP_SNAPSHOT_FINISH } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 137c3f5580cfdd..980a05802da891 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -124,6 +124,9 @@ public abstract class AbstractSnapshotSelfTest extends GridCommonAbstractTest { discoSpi.setIpFinder(((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder()); + if (dfltCacheCfg != null) + cfg.setCacheConfiguration(dfltCacheCfg); + return cfg.setConsistentId(igniteInstanceName) .setCommunicationSpi(new TestRecordingCommunicationSpi()) .setDataStorageConfiguration(new DataStorageConfiguration() diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java index f815b3eda35eb4..9f7f0f076ec29e 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryType; +import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; @@ -40,22 +41,30 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteClusterReadOnlyException; +import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; +import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; +import org.jetbrains.annotations.Nullable; +import org.junit.Ignore; import org.junit.Test; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.RESTORE_GRP_KEY; +import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_FINISH; 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.runAsync; /** @@ -80,7 +89,7 @@ public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTes if (cacheCfgs != null) cfg.setCacheConfiguration(cacheCfgs); - else { + else if (dfltCacheCfg != null) { dfltCacheCfg.setSqlIndexMaxInlineSize(255); dfltCacheCfg.setQueryEntities( Arrays.asList(queryEntity(BIN_TYPE_NAME), queryEntity(IndexedObject.class.getName()))); @@ -100,15 +109,44 @@ private QueryEntity queryEntity(String typeName) { .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); } + /** {@inheritDoc} */ + @Override public void afterTestSnapshot() throws Exception { + try { + for (Ignite ignite : G.allGrids()) { + IgniteEx grid = (IgniteEx)ignite; + + if (grid.context().clientNode()) + continue; + + MetaStorage metaStorage = grid.context().cache().context().database().metaStorage(); + + if (metaStorage == null) + continue; + + Object val = metaStorage.read(RESTORE_GRP_KEY); + + assertNull("Metastorage key has not beend removed [node=" + grid.localNode().id() + ']', val); + } + } finally { + super.afterTestSnapshot(); + } + } + + // @Test +// public void computeTest() throws Exception { +// IgniteEx node = startGrid(4); +// +// node.context().task().execute() +// } + /** @throws Exception If failed. */ @Test public void testBasicClusterSnapshotRestore() throws Exception { int keysCnt = 10_000; - IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); + IgniteEx ignite = startGridsWithSnapshot(2, keysCnt, true); - ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); @@ -133,8 +171,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { forceCheckpoint(); - ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -161,6 +198,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception /** @throws Exception If failed. */ @Test + @Ignore public void testRestoreWithMissedPartitions() throws Exception { IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, dfltCacheCfg.setBackups(0)); @@ -178,7 +216,7 @@ public void testRestoreWithMissedPartitions() throws Exception { ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( - log, () -> fut.get(TIMEOUT), IgniteCheckedException.class, "not all partitions available"); + log, () -> fut.get(TIMEOUT), IgniteException.class, "not all partitions available"); startGrid(1); @@ -276,7 +314,7 @@ public void testRestoreSharedCacheGroup() throws Exception { log, () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(TIMEOUT), IllegalArgumentException.class, - "Cache group(s) not found in snapshot" + "Cache group(s) was not found in the snapshot" ); cache2.destroy(); @@ -324,6 +362,10 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { forceCheckpoint(); + awaitPartitionMapExchange(); + + U.sleep(2_000); + // After destroying the cache with a node filter, the configuration file remains on the filtered node. // todo https://issues.apache.org/jira/browse/IGNITE-14044 for (String cacheName : new String[] {cacheName1, cacheName2}) { @@ -331,8 +373,6 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { U.delete(resolveCacheDir(grid(nodeIdx), cacheName)); } - ignite0.cluster().state(ClusterState.ACTIVE); - ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); awaitPartitionMapExchange(); @@ -403,7 +443,7 @@ public void testIncompatibleMetasUpdate() throws Exception { log, () -> fut0.get(TIMEOUT), IgniteException.class, - "Cache group restore operation was rejected. Incompatible binary types found" + "Incompatible binary types found" ); ensureCacheDirEmpty(2, dfltCacheCfg.getName()); @@ -431,27 +471,41 @@ private IgniteCache createCacheWithBinaryType( return cache; } +// /** +// * @throws Exception if failed +// */ +// @Test +// public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { +// checkCacheStartWithTheSameName(true); +// } + /** * @throws Exception if failed */ @Test public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { - checkCacheStartWithTheSameName(true); + checkCacheStartWithTheSameName(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, IgniteCheckedException.class, + "Cache start failed. A cache named \"cache1\" is currently being restored from a snapshot."); } /** * @throws Exception if failed */ @Test - public void testParallelCacheStartWithTheSameNameOnPerform() throws Exception { - checkCacheStartWithTheSameName(false); + public void testParallelCacheStartWithTheSameNameOnStart() throws Exception { + checkCacheStartWithTheSameName(RESTORE_CACHE_GROUP_SNAPSHOT_START, CacheExistsException.class, + "Failed to start cache (a cache with the same name is already started):"); } /** - * @param prepare {@code True} to start cache during prepare phase, {@code False} to start cache during perform phase. + * @param procType The type of distributed process on which communication is blocked. * @throws Exception if failed. */ - private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { + private void checkCacheStartWithTheSameName( + DistributedProcessType procType, + Class expCls, + String expMsg + ) throws Exception { String grpName = "shared"; String cacheName = "cache1"; @@ -461,48 +515,16 @@ private void checkCacheStartWithTheSameName(boolean prepare) throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, prepare ? - RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE : RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, grpName); + IgniteFuture fut = waitForBlockOnRestore(spi, procType, grpName); - String msgFormat = "Cache start failed. A cache named \"%s\" is currently being restored from a snapshot."; - - GridTestUtils.assertThrowsAnyCause( - log, - () -> ignite.createCache(grpName), - IgniteCheckedException.class, - String.format(msgFormat, grpName) - ); - - if (prepare) - ignite.createCache(cacheName); - else { - GridTestUtils.assertThrowsAnyCause( - log, - () -> ignite.createCache(cacheName), - IgniteCheckedException.class, - String.format(msgFormat, cacheName) - ); - } + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(grpName), IgniteCheckedException.class, null); + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); spi.stopBlock(); - // We don't know shared cache names during prepare phase - so we just interrupting process. - if (prepare) { - GridTestUtils.assertThrowsAnyCause( - log, - () -> fut.get(TIMEOUT), - IgniteException.class, - "Cache \"" + cacheName + "\" should be destroyed manually before perform restore operation." - ); - - ensureCacheDirEmpty(0, grpName); - ensureCacheDirEmpty(1, grpName); - } - else { - fut.get(TIMEOUT); + fut.get(TIMEOUT); - checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); - } + checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -528,19 +550,33 @@ private void checkBaselineChange(boolean stopNode) throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); - IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, dfltCacheCfg.getName()); if (stopNode) { - runAsync(() -> stopGrid(3, true)); + IgniteInternalFuture fut0 = runAsync(() -> stopGrid(3, true)); GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(TIMEOUT), IgniteException.class, - "Cache group restore operation was rejected. Baseline node has left the cluster" + "Cache group restore operation was rejected. Baseline node(s) has left the cluster" ); - ensureCacheDirEmpty(3, dfltCacheCfg.getName()); + fut0.get(TIMEOUT); + + String cacheName = dfltCacheCfg.getName(); + + awaitPartitionMapExchange(); + + dfltCacheCfg = null; + + startGrid(3); + + resetBaselineTopology(); + + awaitPartitionMapExchange(); + + ensureCacheDirEmpty(4, cacheName); return; } @@ -567,85 +603,130 @@ private void checkBaselineChange(boolean stopNode) throws Exception { * @throws Exception if failed. */ @Test - public void testClusterStateChangeActiveReadonlyDuringPrepare() throws Exception { - checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); + public void testClusterStateChangeActiveReadonlyOnPrepare() throws Exception { + checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, + IgniteCheckedException.class, + "Cluster state has been changed"); } - /** * @throws Exception if failed. */ @Test - public void testClusterStateChangeActiveReadonlyDuringPerform() throws Exception { - checkReadOnlyDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); + public void testClusterStateChangeActiveReadonlyOnCacheStart() throws Exception { + checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_START, null, null); } +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testClusterStateChangeActiveReadonlyOnFinish() throws Exception { +// checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, null, null); +// } + /** - * @param procType The type of distributed process on which communication is blocked. * @throws Exception if failed. */ - private void checkReadOnlyDuringRestoring(DistributedProcessType procType) throws Exception { - checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, procType, IgniteClusterReadOnlyException.class, - "Failed to perform start cache operation (cluster is in read-only mode)"); + @Test + public void testClusterDeactivateOnPrepare() throws Exception { + checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, + IgniteCheckedException.class, "Cluster state has been changed."); } /** * @throws Exception if failed. */ @Test - public void testClusterDeactivateOnPrepare() throws Exception { - checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE); + public void testClusterDeactivateOnCacheStart() throws Exception { + checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_START, null, null); } /** * @throws Exception if failed. */ @Test - public void testClusterDeactivateOnPerform() throws Exception { - checkDeactivationDuringRestoring(RESTORE_CACHE_GROUP_SNAPSHOT_PERFORM); + @Ignore + public void testClusterDeactivateOnFinish() throws Exception { + checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, + IgniteException.class, "Baseline node(s) has left the cluster", true); } /** + * @param state Cluster state. * @param procType The type of distributed process on which communication is blocked. + * @param exCls Expected exception class. + * @param expMsg Expected exception message. * @throws Exception if failed. */ - private void checkDeactivationDuringRestoring(DistributedProcessType procType) throws Exception { - checkClusterStateChange(ClusterState.INACTIVE, procType, IgniteCheckedException.class, - "The cluster has been deactivated."); + private void checkClusterStateChange( + ClusterState state, + DistributedProcessType procType, + @Nullable Class exCls, + @Nullable String expMsg + ) throws Exception { + checkClusterStateChange(state, procType, exCls, expMsg, false); } /** * @param state Cluster state. * @param procType The type of distributed process on which communication is blocked. - * @param expCls Expected exception class. + * @param exCls Expected exception class. * @param expMsg Expected exception message. + * @param stopNode Stop node flag. * @throws Exception if failed. */ private void checkClusterStateChange( ClusterState state, DistributedProcessType procType, - Class expCls, - String expMsg + @Nullable Class exCls, + @Nullable String expMsg, + boolean stopNode ) throws Exception { - Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + int nodesCnt = stopNode ? 3 : 2; - TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + Ignite ignite = startGridsWithSnapshot(nodesCnt, CACHE_KEYS_RANGE, true); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(nodesCnt - 1)); IgniteFuture fut = waitForBlockOnRestore(spi, procType, dfltCacheCfg.getName()); ignite.cluster().state(state); - spi.stopBlock(); + if (stopNode) + stopGrid(nodesCnt - 1); + else + spi.stopBlock(); - GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), expCls, expMsg); + if (exCls == null) { + fut.get(TIMEOUT); - ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + ignite.cluster().state(ClusterState.ACTIVE); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + + return; + } + + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), exCls, expMsg); ignite.cluster().state(ClusterState.ACTIVE); - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + ensureCacheDirEmpty(stopNode ? nodesCnt - 1 : nodesCnt, dfltCacheCfg.getName()); - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + String cacheName = dfltCacheCfg.getName(); + + if (stopNode) { + dfltCacheCfg = null; + + startGrid(nodesCnt - 1); + + resetBaselineTopology(); + } + + grid(nodesCnt - 1).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); + + checkCacheKeys(ignite.cache(cacheName), CACHE_KEYS_RANGE); } /** @@ -657,6 +738,10 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { IgniteEx grid = grid(nodeIdx); + CacheGroupDescriptor desc = grid.context().cache().cacheGroupDescriptors().get(CU.cacheId(cacheName)); + + assertNull("nodeIdx=" + nodeIdx + ", cache=" + cacheName, desc); + GridTestUtils.waitForCondition( () -> !grid.context().cache().context().snapshotMgr().isCacheRestoring(null), TIMEOUT); @@ -696,7 +781,21 @@ private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws Igni * @throws Exception if failed. */ private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { - IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valBuilder, dfltCacheCfg); + return startGridsWithSnapshot(nodesCnt, keysCnt, false); + } + + /** + * @param nodesCnt Nodes count. + * @param keysCnt Number of keys to create. + * @param startClient {@code True} to start an additional client node. + * @return Ignite coordinator instance. + * @throws Exception if failed. + */ + private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt, boolean startClient) throws Exception { + IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valBuilder, dfltCacheCfg.setBackups(0)); + + if (startClient) + ignite = startClientGrid("client"); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); From 7a1633dedfeb6c80531768f293c81c3b3f92bd8c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Sun, 14 Feb 2021 22:08:05 +0300 Subject: [PATCH 25/98] IGNITE-13805 Code cleanup. --- .../processors/cache/CachesRegistry.java | 2 +- .../SnapshotRestoreCacheGroupProcess.java | 15 +++++---------- .../snapshot/SnapshotRestoreContext.java | 17 +++++++++++++---- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java index 37d3b2b5bc3a80..6010cc1d627178 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CachesRegistry.java @@ -186,7 +186,7 @@ public IgniteInternalFuture update(ExchangeActions exchActions) { for (ExchangeActions.CacheGroupActionData stopAction : exchActions.cacheGroupsToStop()) { CacheGroupDescriptor rmvd = unregisterGroup(stopAction.descriptor().groupId()); -// assert rmvd != null : stopAction.descriptor().cacheOrGroupName(); + assert rmvd != null : stopAction.descriptor().cacheOrGroupName(); } for (ExchangeActions.CacheActionData req : exchActions.cacheStopRequests()) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 0f920e700f0008..c37a94f2660679 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -227,9 +227,7 @@ public boolean staleFuture(IgniteInternalFuture fut) { * @param leftNodeId Left node ID. */ public void onNodeLeft(UUID leftNodeId) { - IgniteInternalFuture fut0 = fut; - - if (staleFuture(fut0)) + if (staleFuture(fut)) return; if (opCtx.nodes().contains(leftNodeId)) { @@ -386,21 +384,18 @@ private void finishPrepare(UUID reqId, Map res, M private IgniteInternalFuture cacheStart(SnapshotRestoreRequest req) { SnapshotRestoreContext opCtx0 = opCtx; - if (staleFuture(fut) || !req.requestId().equals(opCtx0.requestId())) - return new GridFinishedFuture<>(); - - if (!opCtx0.nodes().contains(ctx.localNodeId())) + if (staleFuture(fut)) return new GridFinishedFuture<>(); if (!req.requestId().equals(opCtx0.requestId())) return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); - if (ctx.state().clusterState().state() != ClusterState.ACTIVE) - return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Cluster state has been changed.")); - if (!U.isLocalNodeCoordinator(ctx.discovery())) return new GridFinishedFuture<>(); + if (ctx.state().clusterState().state() != ClusterState.ACTIVE) + return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Cluster state has been changed.")); + if (interrupted()) return new GridFinishedFuture<>(errRef.get()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index af711d100dceea..03f898e0c92703 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -59,8 +60,11 @@ class SnapshotRestoreContext { /** Restore operation lock. */ private final ReentrantLock rollbackLock = new ReentrantLock(); + /** List of processed cache IDs. */ + private final Set cacheIds = new HashSet<>(); + /** Cache configurations. */ - private final Map cacheCfgs = new ConcurrentHashMap<>(); + private final List ccfgs; /** Restored cache groups. */ private final Map grps = new ConcurrentHashMap<>(); @@ -74,15 +78,20 @@ class SnapshotRestoreContext { */ public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List configs, GridKernalContext ctx) { + ccfgs = new ArrayList<>(configs); + for (StoredCacheData cacheData : configs) { String cacheName = cacheData.config().getName(); - cacheCfgs.put(CU.cacheId(cacheName), cacheData); + cacheIds.add(CU.cacheId(cacheName)); boolean shared = cacheData.config().getGroupName() != null; grps.computeIfAbsent( shared ? cacheData.config().getGroupName() : cacheName, v -> new GroupRestoreContext(shared)); + + if (shared) + cacheIds.add(CU.cacheId(cacheData.config().getGroupName())); } this.reqId = reqId; @@ -126,12 +135,12 @@ public Collection groupDirs() { * @return {@code True} if the cache with the specified name is currently being restored. */ public boolean containsCache(String name) { - return grps.containsKey(name) || cacheCfgs.containsKey(CU.cacheId(name)); + return cacheIds.contains(CU.cacheId(name)); } /** @return Cache configurations. */ public Collection configs() { - return cacheCfgs.values(); + return ccfgs; } /** From d39a6610c29fcdd6d6e22e169e027d3de2bee24b Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 16 Feb 2021 21:19:41 +0300 Subject: [PATCH 26/98] IGNITE-13805 Fail cache start if node left during cache start. --- .../discovery/GridDiscoveryManager.java | 3 +- .../processors/cache/ClusterCachesInfo.java | 57 +++++++++++++------ .../cache/DynamicCacheChangeBatch.java | 11 +++- .../processors/cache/ExchangeActions.java | 11 ++++ .../processors/cache/GridCacheProcessor.java | 54 ++++++++++++++++-- .../GridDhtPartitionsExchangeFuture.java | 22 ++++++- .../SnapshotRestoreCacheGroupProcess.java | 10 ++-- ...IgniteClusterSnapshotRestoreSelfTest.java} | 42 +++++++++++--- .../IgnitePdsWithIndexingTestSuite.java | 4 +- 9 files changed, 169 insertions(+), 45 deletions(-) rename modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{IgniteClusterSnapshoRestoreSelfTest.java => IgniteClusterSnapshotRestoreSelfTest.java} (96%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 39b4c5a989611b..cf1f72da90582a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -638,7 +638,8 @@ else if (customMsg instanceof ChangeGlobalStateFinishMessage) { incMinorTopVer = ctx.cache().onCustomEvent( customMsg, new AffinityTopologyVersion(topVer, minorTopVer), - node); + node, + notification.getTopSnapshot()); } if (incMinorTopVer) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index d074ff4d1e8c61..823a07c6d27567 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteNodeAttributes; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.managers.encryption.GridEncryptionManager; @@ -577,28 +578,13 @@ public void onCacheChangeRequested(DynamicCacheChangeFailureMessage failMsg, Aff /** * @param batch Cache change request. * @param topVer Topology version. + * @param topSnapshot Topology snapshot. * @return {@code True} if minor topology version should be increased. */ - public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) { + public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer, Collection topSnapshot) { DiscoveryDataClusterState state = ctx.state().clusterState(); - if (state.active() && !state.transition()) { - ExchangeActions exchangeActions = new ExchangeActions(); - - CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions, - batch.requests(), - topVer, - false); - - if (res.needExchange) { - assert !exchangeActions.empty() : exchangeActions; - - batch.exchangeActions(exchangeActions); - } - - return res.needExchange; - } - else { + if (!state.active() || state.transition()) { IgniteCheckedException err = new IgniteCheckedException("Failed to start/stop cache, cluster state change " + "is in progress."); @@ -613,6 +599,41 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop return false; } + + if (!F.isEmpty(batch.nodes())) { + Set srvNodes = + new HashSet<>(F.viewReadOnly(topSnapshot, ClusterNode::id, n -> !n.isClient() && !n.isDaemon())); + + if (!srvNodes.containsAll(batch.nodes()) || !ctx.discovery().aliveAll(batch.nodes())) { + ClusterTopologyCheckedException err = + new ClusterTopologyCheckedException("Server node(s) has left the cluster."); + + for (DynamicCacheChangeRequest req : batch.requests()) + ctx.cache().completeCacheStartFuture(req, false, err); + + return false; + } + } + + ExchangeActions exchangeActions = new ExchangeActions(); + + CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions, + batch.requests(), + topVer, + false); + + if (res.needExchange) { + assert !exchangeActions.empty() : exchangeActions; + + batch.exchangeActions(exchangeActions); + + if (!F.isEmpty(batch.nodes())) { + U.dumpStack(">xxx> set failCacheStartOnNodeLeft=true"); + exchangeActions.failCacheStartOnNodeLeft(true); + } + } + + return res.needExchange; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index 6b89424462b0d6..69221673ffc43c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Set; +import java.util.UUID; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -45,6 +46,8 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { @GridToStringInclude private Collection reqs; + private Collection reqNodes; + /** Cache updates to be executed on exchange. */ private transient ExchangeActions exchangeActions; @@ -60,11 +63,13 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { /** * @param reqs Requests. + * @param reqNodes todo */ - public DynamicCacheChangeBatch(Collection reqs) { + public DynamicCacheChangeBatch(Collection reqs, Collection reqNodes) { assert !F.isEmpty(reqs) : reqs; this.reqs = reqs; + this.reqNodes = reqNodes; } /** {@inheritDoc} */ @@ -162,6 +167,10 @@ public void startCaches(boolean startCaches) { this.startCaches = startCaches; } + public Collection nodes() { + return reqNodes; + } + /** {@inheritDoc} */ @Override public String toString() { return S.toString(DynamicCacheChangeBatch.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java index cbe7df41b81ab1..c05677d23a4561 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -43,6 +43,9 @@ public class ExchangeActions { /** */ private Map cachesToStart; + /** */ + private boolean failCacheStartOnNodeLeft; + /** */ private Map cachesToStop; @@ -319,6 +322,14 @@ public boolean cacheGroupStarting(int grpId) { return false; } + public boolean failCacheStartOnNodeLeft() { + return failCacheStartOnNodeLeft; + } + + public void failCacheStartOnNodeLeft(boolean failCacheStartOnNodeLeft) { + this.failCacheStartOnNodeLeft = failCacheStartOnNodeLeft; + } + /** * @param grpDesc Group descriptor. * @param destroy Destroy flag. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 4b1b508716c226..1fcf176da12409 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -227,6 +227,8 @@ */ @SuppressWarnings({"unchecked", "TypeMayBeWeakened", "deprecation"}) public class GridCacheProcessor extends GridProcessorAdapter { + public static final CountDownLatch MEGALATCH = new CountDownLatch(3); + /** */ public static final String CLUSTER_READ_ONLY_MODE_ERROR_MSG_FORMAT = "Failed to perform %s operation (cluster is in read-only mode) [cacheGrp=%s, cache=%s]"; @@ -1991,6 +1993,20 @@ private GridCacheContext prepareCacheContext( AffinityTopologyVersion exchTopVer, boolean disabledAfterStart ) throws IgniteCheckedException { + if ("SuperCache".equals(desc.cacheName())) + try { + U.sleep(5_000); + } + catch (IgniteInterruptedCheckedException e) { + Thread.currentThread().interrupt(); + + throw new IgniteException(e); + } + + MEGALATCH.countDown(); + + U.dumpStack(">xxx> prepare cache context " + desc.cacheName()); + desc = enricher().enrich(desc, desc.cacheConfiguration().getCacheMode() == LOCAL || isLocalAffinity(desc.cacheConfiguration())); @@ -3727,6 +3743,18 @@ public IgniteInternalFuture dynamicStartCaches( restoredCache); } + public IgniteInternalFuture dynamicStartCachesByStoredConf( + Collection storedCacheDataList, + boolean failIfExists, + boolean checkThreadTx, + boolean disabledAfterStart, + IgniteUuid restartId, + boolean restoredCache + ) { + return dynamicStartCachesByStoredConf(storedCacheDataList, failIfExists, checkThreadTx, disabledAfterStart, + restartId, restoredCache, null); + } + /** * Dynamically starts multiple caches. * @@ -3743,7 +3771,8 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean checkThreadTx, boolean disabledAfterStart, IgniteUuid restartId, - boolean restoredCache + boolean restoredCache, + @Nullable Set reqNodes ) { if (checkThreadTx) { sharedCtx.tm().checkEmptyTransactions(() -> { @@ -3805,7 +3834,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( GridCompoundFuture compoundFut = new GridCompoundFuture<>(); - for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs)) + for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs, reqNodes)) compoundFut.add((IgniteInternalFuture)fut); if (clientReqs != null) { @@ -4058,6 +4087,18 @@ public void cleanupCachesDirectories() throws IgniteCheckedException { */ private Collection initiateCacheChanges( Collection reqs + ) { + return initiateCacheChanges(reqs, null); + } + + /** + * @param reqs Requests. + * @param reqNodes todo + * @return Collection of futures. + */ + private Collection initiateCacheChanges( + Collection reqs, + Collection reqNodes ) { Collection res = new ArrayList<>(reqs.size()); @@ -4113,7 +4154,7 @@ private Collection initiateCacheChanges( if (!sndReqs.isEmpty()) { try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs)); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, reqNodes)); err = checkNodeState(); } @@ -4213,9 +4254,10 @@ public void onDiscoveryEvent( * @param msg Customer message. * @param topVer Current topology version. * @param node Node sent message. + * @param topSnapshot Topology snapshot. * @return {@code True} if minor topology version should be increased. */ - public boolean onCustomEvent(DiscoveryCustomMessage msg, AffinityTopologyVersion topVer, ClusterNode node) { + public boolean onCustomEvent(DiscoveryCustomMessage msg, AffinityTopologyVersion topVer, ClusterNode node, Collection topSnapshot) { if (msg instanceof SchemaAbstractDiscoveryMessage) { ctx.query().onDiscovery((SchemaAbstractDiscoveryMessage)msg); @@ -4241,7 +4283,7 @@ else if (msg0 instanceof WalStateFinishMessage) } if (msg instanceof DynamicCacheChangeBatch) { - boolean changeRequested = cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer); + boolean changeRequested = cachesInfo.onCacheChangeRequested((DynamicCacheChangeBatch)msg, topVer, topSnapshot); ctx.query().onCacheChangeRequested((DynamicCacheChangeBatch)msg); @@ -4678,7 +4720,7 @@ public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteChec Exception err = null; try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req))); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req), null)); if (ctx.isStopping()) { err = new IgniteCheckedException("Failed to execute dynamic cache change request, " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index ca051356709ea7..e04110adb0c2c3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -3827,10 +3827,26 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe return; try { - if (!F.isEmpty(exchangeGlobalExceptions) && dynamicCacheStartExchange() && isRollbackSupported()) { - sendExchangeFailureMessage(); + if (dynamicCacheStartExchange() && isRollbackSupported()) { + if (!F.isEmpty(exchangeGlobalExceptions)) { + sendExchangeFailureMessage(); - return; + return; + } + + if (exchActions.failCacheStartOnNodeLeft()) { + Collection srvs = F.viewReadOnly(firstEvent().topologyNodes(), ClusterNode::id, n -> !n.isDaemon() && !n.isClient()); + + if (!cctx.discovery().aliveAll(srvs)) { + exchangeGlobalExceptions.put(cctx.localNodeId(), new ClusterTopologyCheckedException("Server node(s) has left the cluster.")); + + sendExchangeFailureMessage(); + + return; + } + else + System.out.println(">xxx> firstEvent all alive: " + srvs); + } } AffinityTopologyVersion resTopVer = exchCtx.events().topologyVersion(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index c37a94f2660679..a533fcfb74d270 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -232,7 +232,7 @@ public void onNodeLeft(UUID leftNodeId) { if (opCtx.nodes().contains(leftNodeId)) { interrupt(new IgniteException(OP_REJECT_MSG + - "Baseline node(s) has left the cluster [nodeId=" + leftNodeId + ']')); + "Server node(s) has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -294,7 +294,7 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq if (!baselineNodes().containsAll(req.nodes())) { return new GridFinishedFuture<>( - new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster.")); + new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); } SnapshotRestoreContext opCtx0 = opCtx; @@ -400,7 +400,7 @@ private IgniteInternalFuture cacheStart(SnapshotRestore return new GridFinishedFuture<>(errRef.get()); if (!baselineNodes().containsAll(opCtx0.nodes())) - return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster.")); + return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -410,7 +410,7 @@ private IgniteInternalFuture cacheStart(SnapshotRestore ", caches=" + F.viewReadOnly(opCtx0.configs(), c -> c.config().getName()) + ']'); } - ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true).listen( + ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true, opCtx0.nodes()).listen( f -> { if (f.error() != null) { log.error("Unable to start restored caches.", f.error()); @@ -444,7 +444,7 @@ private void finishCacheStart(UUID reqId, Map res leftNodes.removeAll(res.keySet()); - failure = new IgniteException(OP_REJECT_MSG + "Baseline node(s) has left the cluster [nodeId=" + leftNodes + ']'); + failure = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); } if (failure != null) { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java similarity index 96% rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java index 9f7f0f076ec29e..08d34884c2d917 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshoRestoreSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreSelfTest.java @@ -43,7 +43,9 @@ import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; +import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; @@ -70,7 +72,7 @@ /** * Snapshot restore tests. */ -public class IgniteClusterSnapshoRestoreSelfTest extends AbstractSnapshotSelfTest { +public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTest { /** Timeout. */ private static final long TIMEOUT = 15_000; @@ -132,12 +134,34 @@ private QueryEntity queryEntity(String typeName) { } } - // @Test -// public void computeTest() throws Exception { -// IgniteEx node = startGrid(4); -// -// node.context().task().execute() -// } + /** + * Ensures that the cache doesn't start if one of the baseline nodes fails. + * + * @throws Exception If failed. + */ + @Test + public void testCacheStartFailOnNodeLeft() throws Exception { + int keysCnt = 10_000; + + startGridsWithSnapshot(3, keysCnt, true); + + BlockingCustomMessageDiscoverySpi discoSpi = discoSpi(grid(0)); + + discoSpi.block((msg) -> msg instanceof DynamicCacheChangeBatch); + + IgniteFuture fut = + grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + discoSpi.waitBlocked(TIMEOUT); + + stopGrid(2, true); + + discoSpi.unblock(); + + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, null); + + ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + } /** @throws Exception If failed. */ @Test @@ -559,7 +583,7 @@ private void checkBaselineChange(boolean stopNode) throws Exception { log, () -> fut.get(TIMEOUT), IgniteException.class, - "Cache group restore operation was rejected. Baseline node(s) has left the cluster" + "Cache group restore operation was rejected. Server node(s) has left the cluster" ); fut0.get(TIMEOUT); @@ -649,7 +673,7 @@ public void testClusterDeactivateOnCacheStart() throws Exception { @Ignore public void testClusterDeactivateOnFinish() throws Exception { checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, - IgniteException.class, "Baseline node(s) has left the cluster", true); + IgniteException.class, "Server node(s) has left the cluster", true); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java index 72f81a8465f494..9e5d299200cb9b 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java @@ -28,8 +28,8 @@ import org.apache.ignite.internal.processors.cache.persistence.db.IndexingMultithreadedLoadContinuousRestartTest; import org.apache.ignite.internal.processors.cache.persistence.db.LongDestroyDurableBackgroundTaskTest; import org.apache.ignite.internal.processors.cache.persistence.db.MultipleParallelCacheDeleteDeadlockTest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshoRestoreSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotCheckWithIndexesTest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotRestoreSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotWithIndexesTest; import org.apache.ignite.internal.processors.database.IgniteDbMultiNodeWithIndexingPutGetTest; import org.apache.ignite.internal.processors.database.IgniteDbSingleNodeWithIndexingPutGetTest; @@ -68,7 +68,7 @@ IgnitePdsIndexingDefragmentationTest.class, StopRebuildIndexTest.class, ForceRebuildIndexTest.class, - IgniteClusterSnapshoRestoreSelfTest.class + IgniteClusterSnapshotRestoreSelfTest.class }) public class IgnitePdsWithIndexingTestSuite { } From 1eaa78d52f3f9accd52fb6c4755d74b94a1d08f7 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 17 Feb 2021 22:41:06 +0300 Subject: [PATCH 27/98] IGNITE-13805 Don't cleanup directories in automatic mode. --- .../processors/cache/GridCacheProcessor.java | 2 - .../snapshot/IgniteSnapshotManager.java | 66 +------------------ .../SnapshotRestoreCacheGroupProcess.java | 12 ---- .../snapshot/SnapshotRestoreContext.java | 38 ++--------- .../IgniteClusterSnapshotRestoreSelfTest.java | 59 ++++------------- 5 files changed, 19 insertions(+), 158 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 1fcf176da12409..b07b074e9fa35b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -5481,8 +5481,6 @@ private class CacheRecoveryLifecycle implements MetastorageLifecycleListener, Da /** {@inheritDoc} */ @Override public void onReadyForRead(ReadOnlyMetastorage metastorage) throws IgniteCheckedException { - ctx.cache().context().snapshotMgr().cleanupRestoredCacheGroups(metastorage); - CacheJoinNodeDiscoveryData data = locCfgMgr.restoreCacheConfigurations(); cachesInfo.onStart(data); 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 a30f3520a7a9ef..8130b9b74a27ff 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 @@ -219,9 +219,6 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** Metastorage key to save currently running snapshot. */ public static final String SNP_RUNNING_KEY = "snapshot-running"; - /** Metastorage key to save currently restoring cache groups. */ - public static final String RESTORE_GRP_KEY = "snapshotRestoreGroups"; - /** Snapshot metrics prefix. */ public static final String SNAPSHOT_METRICS = "snapshot"; @@ -308,9 +305,6 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter /** {@code true} if recovery process occurred for snapshot. */ private volatile boolean recovered; - /** {@code true} if recovery process occurred for snapshot restore. */ - private volatile boolean restoreRecovered; - /** Last seen cluster snapshot operation. */ private volatile ClusterSnapshotFuture lastSeenSnpFut = new ClusterSnapshotFuture(); @@ -472,17 +466,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onActivate(GridKernalContext kctx) { - try { - if (metaStorage == null) - return; - - // Remove metastorage key used for recovery in case cluster has been - // deactivated and process was not able to do this. - updateRecoveryDataForRestoredGroups(null); - } - catch (IgniteCheckedException e) { - log.warning("Unable to remove key from metastorage.", e); - } + // No-op. } /** {@inheritDoc} */ @@ -1178,27 +1162,6 @@ protected void restoreCacheGroupFiles( } } - /** - * @param dirs Cache group directory names, required to clean up in case of node failure during the restore process. - * @throws IgniteCheckedException If failed. - */ - protected void updateRecoveryDataForRestoredGroups(@Nullable List dirs) throws IgniteCheckedException { - if (!cctx.kernalContext().state().clusterState().state().active()) - throw new IgniteCheckedException("Unable to update key in metastorage - cluster is not active."); - - cctx.database().checkpointReadLock(); - - try { - if (dirs == null) - metaStorage.remove(RESTORE_GRP_KEY); - else - metaStorage.write(RESTORE_GRP_KEY, new ArrayList<>(dirs)); - } - finally { - cctx.database().checkpointReadUnlock(); - } - } - /** * @param files Collection of files to delete. */ @@ -1249,11 +1212,6 @@ public File resolveSnapshotCacheDir(String snpName, String cacheName) { recovered = false; } - - if (restoreRecovered) - updateRecoveryDataForRestoredGroups(null); - - restoreRecovered = false; } /** {@inheritDoc} */ @@ -1277,28 +1235,6 @@ public File resolveSnapshotCacheDir(String snpName, String cacheName) { } } - /** - * Clean up restored cache group folders if a node failed during restore process. - * - * @param metaStorage Read-only meta storage. - * @throws IgniteCheckedException If failed. - */ - public void cleanupRestoredCacheGroups(ReadOnlyMetastorage metaStorage) throws IgniteCheckedException { - Collection grps = (Collection)metaStorage.read(RESTORE_GRP_KEY); - - if (grps == null) - return; - - for (String grpDirName : grps) { - File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), DFLT_STORE_DIR + - File.separator + pdsSettings.folderName() + File.separator + grpDirName, false); - - U.delete(cacheDir); - } - - restoreRecovered = true; - } - /** * @param evt Discovery event to check. * @return {@code true} if exchange started by snapshot operation. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index a533fcfb74d270..ab7b36d35cbf19 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,7 +17,6 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; @@ -310,17 +309,6 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq ensureCacheAbsent(cfg.config().getName()); } - fut.listen(f -> { - try { - ctx.cache().context().snapshotMgr().updateRecoveryDataForRestoredGroups(null); - } - catch (IgniteCheckedException e) { - log.warning("Unable to reset restored groups.", e); - } - }); - - ctx.cache().context().snapshotMgr().updateRecoveryDataForRestoredGroups(new ArrayList<>(opCtx0.groupDirs())); - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx0.snapshotName()).exists()) return new GridFinishedFuture<>(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index 03f898e0c92703..2c1212c2c4f1c5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -33,13 +33,10 @@ import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; -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.S; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; /** * Cache group restore from snapshot operation context. @@ -67,7 +64,7 @@ class SnapshotRestoreContext { private final List ccfgs; /** Restored cache groups. */ - private final Map grps = new ConcurrentHashMap<>(); + private final Map> grps = new ConcurrentHashMap<>(); /** * @param reqId Request ID. @@ -87,8 +84,7 @@ public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Li boolean shared = cacheData.config().getGroupName() != null; - grps.computeIfAbsent( - shared ? cacheData.config().getGroupName() : cacheName, v -> new GroupRestoreContext(shared)); + grps.computeIfAbsent(shared ? cacheData.config().getGroupName() : cacheName, v -> new ArrayList<>()); if (shared) cacheIds.add(CU.cacheId(cacheData.config().getGroupName())); @@ -122,14 +118,6 @@ public Set groups() { return grps.keySet(); } - /** - * @return Names of the directories of the restored caches. - */ - public Collection groupDirs() { - return F.viewReadOnly(grps.entrySet(), - e -> (e.getValue().shared ? CACHE_GRP_DIR_PREFIX : CACHE_DIR_PREFIX) + e.getKey()); - } - /** * @param name Cache name. * @return {@code True} if the cache with the specified name is currently being restored. @@ -170,9 +158,7 @@ public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws rollbackLock.lock(); try { - GroupRestoreContext grp = grps.get(grpName); - - ctx.cache().context().snapshotMgr().restoreCacheGroupFiles(snpName, grpName, stopChecker, grp.files); + ctx.cache().context().snapshotMgr().restoreCacheGroupFiles(snpName, grpName, stopChecker, grps.get(grpName)); } finally { rollbackLock.unlock(); @@ -190,10 +176,10 @@ public void rollback() { List grpNames = new ArrayList<>(groups()); for (String grpName : grpNames) { - GroupRestoreContext grp = grps.remove(grpName); + List files = grps.remove(grpName); - if (grp != null) - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(grp.files); + if (files != null) + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); } } finally { @@ -205,16 +191,4 @@ public void rollback() { @Override public String toString() { return S.toString(SnapshotRestoreContext.class, this); } - - /** */ - private static class GroupRestoreContext { - /** Files created in the cache group folder during a restore operation. */ - final List files = new ArrayList<>(); - - final boolean shared; - - private GroupRestoreContext(boolean shared) { - this.shared = shared; - } - } } 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 08d34884c2d917..13243f8a5b44e7 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 @@ -46,11 +46,9 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; -import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; @@ -63,7 +61,6 @@ import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.RESTORE_GRP_KEY; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_FINISH; 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; @@ -111,29 +108,6 @@ private QueryEntity queryEntity(String typeName) { .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); } - /** {@inheritDoc} */ - @Override public void afterTestSnapshot() throws Exception { - try { - for (Ignite ignite : G.allGrids()) { - IgniteEx grid = (IgniteEx)ignite; - - if (grid.context().clientNode()) - continue; - - MetaStorage metaStorage = grid.context().cache().context().database().metaStorage(); - - if (metaStorage == null) - continue; - - Object val = metaStorage.read(RESTORE_GRP_KEY); - - assertNull("Metastorage key has not beend removed [node=" + grid.localNode().id() + ']', val); - } - } finally { - super.afterTestSnapshot(); - } - } - /** * Ensures that the cache doesn't start if one of the baseline nodes fails. * @@ -495,14 +469,6 @@ private IgniteCache createCacheWithBinaryType( return cache; } -// /** -// * @throws Exception if failed -// */ -// @Test -// public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { -// checkCacheStartWithTheSameName(true); -// } - /** * @throws Exception if failed */ @@ -553,21 +519,21 @@ private void checkCacheStartWithTheSameName( /** @throws Exception If failed. */ @Test - public void testRollbackOnNodeFail() throws Exception { - checkBaselineChange(true); + public void testNodeFail() throws Exception { + checkTopologyChange(true); } /** @throws Exception If failed. */ @Test public void testNodeJoin() throws Exception { - checkBaselineChange(false); + checkTopologyChange(false); } /** * @param stopNode {@code True} to check node fail, {@code False} to check node join. * @throws Exception if failed. */ - private void checkBaselineChange(boolean stopNode) throws Exception { + private void checkTopologyChange(boolean stopNode) throws Exception { int keysCnt = 10_000; IgniteEx ignite = startGridsWithSnapshot(4, keysCnt); @@ -586,21 +552,20 @@ private void checkBaselineChange(boolean stopNode) throws Exception { "Cache group restore operation was rejected. Server node(s) has left the cluster" ); - fut0.get(TIMEOUT); + ensureCacheDirEmpty(3, dfltCacheCfg.getName()); - String cacheName = dfltCacheCfg.getName(); + fut0.get(TIMEOUT); awaitPartitionMapExchange(); dfltCacheCfg = null; - startGrid(3); - - resetBaselineTopology(); - - awaitPartitionMapExchange(); - - ensureCacheDirEmpty(4, cacheName); + GridTestUtils.assertThrowsAnyCause( + log, + () -> startGrid(3), + IgniteSpiException.class, + "to add the node to cluster - remove directories with the caches" + ); return; } From 67fe8ed194e485dcbfe4358cb09acebe5dcbab4c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 18 Feb 2021 13:55:14 +0300 Subject: [PATCH 28/98] IGNITE-13805 Code cleanup. --- .../processors/cache/ClusterCachesInfo.java | 12 ++-- .../cache/DynamicCacheChangeBatch.java | 21 ++++-- .../processors/cache/ExchangeActions.java | 21 ++++-- .../processors/cache/GridCacheProcessor.java | 39 +++++------ .../GridDhtPartitionsExchangeFuture.java | 12 ++-- .../SnapshotRestoreCacheGroupProcess.java | 59 +++++------------ .../snapshot/SnapshotRestoreContext.java | 47 +++++++++++--- .../util/distributed/DistributedProcess.java | 2 +- .../IgniteClusterSnapshotRestoreSelfTest.java | 65 +------------------ 9 files changed, 116 insertions(+), 162 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 823a07c6d27567..d51cdf06ddfd0a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -600,11 +600,11 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop return false; } - if (!F.isEmpty(batch.nodes())) { - Set srvNodes = + if (!F.isEmpty(batch.topologyNodes())) { + Set srvNodeIds = new HashSet<>(F.viewReadOnly(topSnapshot, ClusterNode::id, n -> !n.isClient() && !n.isDaemon())); - if (!srvNodes.containsAll(batch.nodes()) || !ctx.discovery().aliveAll(batch.nodes())) { + if (!srvNodeIds.containsAll(batch.topologyNodes()) || !ctx.discovery().aliveAll(batch.topologyNodes())) { ClusterTopologyCheckedException err = new ClusterTopologyCheckedException("Server node(s) has left the cluster."); @@ -627,10 +627,8 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop batch.exchangeActions(exchangeActions); - if (!F.isEmpty(batch.nodes())) { - U.dumpStack(">xxx> set failCacheStartOnNodeLeft=true"); - exchangeActions.failCacheStartOnNodeLeft(true); - } + if (!F.isEmpty(batch.topologyNodes())) + exchangeActions.checkCacheStartTopology(true); } return res.needExchange; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index 69221673ffc43c..dbd97c3f62dacf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -46,7 +46,11 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { @GridToStringInclude private Collection reqs; - private Collection reqNodes; + /** + * Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when starting + * the cache(s), the whole procedure is rolled back. + */ + private Collection topNodes; /** Cache updates to be executed on exchange. */ private transient ExchangeActions exchangeActions; @@ -63,13 +67,14 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { /** * @param reqs Requests. - * @param reqNodes todo + * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails + * when starting the cache(s), the whole procedure is rolled back. */ - public DynamicCacheChangeBatch(Collection reqs, Collection reqNodes) { + public DynamicCacheChangeBatch(Collection reqs, @Nullable Collection topNodes) { assert !F.isEmpty(reqs) : reqs; this.reqs = reqs; - this.reqNodes = reqNodes; + this.topNodes = topNodes; } /** {@inheritDoc} */ @@ -167,8 +172,12 @@ public void startCaches(boolean startCaches) { this.startCaches = startCaches; } - public Collection nodes() { - return reqNodes; + /** + * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when + * starting the cache(s), the whole procedure is rolled back. + */ + public Collection topologyNodes() { + return topNodes; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java index c05677d23a4561..89ced85649bdfc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -43,8 +43,11 @@ public class ExchangeActions { /** */ private Map cachesToStart; - /** */ - private boolean failCacheStartOnNodeLeft; + /** + * Topology check after cache start flag. If this flag is enabled and any of the server nodes is missing after + * starting the cache, the whole procedure fails and is rolled back. + */ + private boolean checkCacheStartTop; /** */ private Map cachesToStop; @@ -322,12 +325,18 @@ public boolean cacheGroupStarting(int grpId) { return false; } - public boolean failCacheStartOnNodeLeft() { - return failCacheStartOnNodeLeft; + /** + * @return Topology check after cache start flag. + */ + public boolean checkCacheStartTopology() { + return checkCacheStartTop; } - public void failCacheStartOnNodeLeft(boolean failCacheStartOnNodeLeft) { - this.failCacheStartOnNodeLeft = failCacheStartOnNodeLeft; + /** + * @param checkCacheStartTop Topology check after cache start flag. + */ + public void checkCacheStartTopology(boolean checkCacheStartTop) { + this.checkCacheStartTop = checkCacheStartTop; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index b07b074e9fa35b..087ab4b753496f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -227,8 +227,6 @@ */ @SuppressWarnings({"unchecked", "TypeMayBeWeakened", "deprecation"}) public class GridCacheProcessor extends GridProcessorAdapter { - public static final CountDownLatch MEGALATCH = new CountDownLatch(3); - /** */ public static final String CLUSTER_READ_ONLY_MODE_ERROR_MSG_FORMAT = "Failed to perform %s operation (cluster is in read-only mode) [cacheGrp=%s, cache=%s]"; @@ -1993,20 +1991,6 @@ private GridCacheContext prepareCacheContext( AffinityTopologyVersion exchTopVer, boolean disabledAfterStart ) throws IgniteCheckedException { - if ("SuperCache".equals(desc.cacheName())) - try { - U.sleep(5_000); - } - catch (IgniteInterruptedCheckedException e) { - Thread.currentThread().interrupt(); - - throw new IgniteException(e); - } - - MEGALATCH.countDown(); - - U.dumpStack(">xxx> prepare cache context " + desc.cacheName()); - desc = enricher().enrich(desc, desc.cacheConfiguration().getCacheMode() == LOCAL || isLocalAffinity(desc.cacheConfiguration())); @@ -3743,6 +3727,16 @@ public IgniteInternalFuture dynamicStartCaches( restoredCache); } + /** + * Dynamically starts multiple caches. + * + * @param storedCacheDataList Collection of stored cache data. + * @param failIfExists Fail if exists flag. + * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. + * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. + * @param restartId Restart requester id (it'll allow to start this cache only him). + * @return Future that will be completed when all caches are deployed. + */ public IgniteInternalFuture dynamicStartCachesByStoredConf( Collection storedCacheDataList, boolean failIfExists, @@ -3763,6 +3757,8 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. * @param restartId Restart requester id (it'll allow to start this cache only him). + * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails + * when starting the cache(s), the whole procedure is rolled back. * @return Future that will be completed when all caches are deployed. */ public IgniteInternalFuture dynamicStartCachesByStoredConf( @@ -3772,7 +3768,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean disabledAfterStart, IgniteUuid restartId, boolean restoredCache, - @Nullable Set reqNodes + @Nullable Set topNodes ) { if (checkThreadTx) { sharedCtx.tm().checkEmptyTransactions(() -> { @@ -3834,7 +3830,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( GridCompoundFuture compoundFut = new GridCompoundFuture<>(); - for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs, reqNodes)) + for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs, topNodes)) compoundFut.add((IgniteInternalFuture)fut); if (clientReqs != null) { @@ -4093,12 +4089,13 @@ private Collection initiateCacheChanges( /** * @param reqs Requests. - * @param reqNodes todo + * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails + * when starting the cache(s), the whole procedure is rolled back. * @return Collection of futures. */ private Collection initiateCacheChanges( Collection reqs, - Collection reqNodes + @Nullable Collection topNodes ) { Collection res = new ArrayList<>(reqs.size()); @@ -4154,7 +4151,7 @@ private Collection initiateCacheChanges( if (!sndReqs.isEmpty()) { try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, reqNodes)); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, topNodes)); err = checkNodeState(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index e04110adb0c2c3..96a3b761976fb5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -3834,18 +3834,18 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe return; } - if (exchActions.failCacheStartOnNodeLeft()) { - Collection srvs = F.viewReadOnly(firstEvent().topologyNodes(), ClusterNode::id, n -> !n.isDaemon() && !n.isClient()); + if (exchActions.checkCacheStartTopology()) { + Collection srvNodeIds = F.viewReadOnly( + firstEvent().topologyNodes(), ClusterNode::id, n -> !n.isDaemon() && !n.isClient()); - if (!cctx.discovery().aliveAll(srvs)) { - exchangeGlobalExceptions.put(cctx.localNodeId(), new ClusterTopologyCheckedException("Server node(s) has left the cluster.")); + if (!cctx.discovery().aliveAll(srvNodeIds)) { + exchangeGlobalExceptions.put(cctx.localNodeId(), + new ClusterTopologyCheckedException("Server node(s) has left the cluster.")); sendExchangeFailureMessage(); return; } - else - System.out.println(">xxx> firstEvent all alive: " + srvs); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index ab7b36d35cbf19..a6fd95489f884f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -24,7 +24,6 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; -import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -47,8 +46,8 @@ import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_FINISH; 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_ROLLBACK; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_START; /** @@ -79,9 +78,6 @@ public class SnapshotRestoreCacheGroupProcess { /** Snapshot restore operation context. */ private volatile SnapshotRestoreContext opCtx; - /** The exception that led to the interruption of the process. */ - private final AtomicReference errRef = new AtomicReference<>(); - /** * @param ctx Kernal context. */ @@ -97,7 +93,7 @@ public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { ctx, RESTORE_CACHE_GROUP_SNAPSHOT_START, this::cacheStart, this::finishCacheStart); rollbackRestoreProc = new DistributedProcess<>( - ctx, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, this::rollback, this::finishRollback); + ctx, RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK, this::rollback, this::finishRollback); fut.onDone(); } @@ -195,23 +191,6 @@ public boolean inProgress(@Nullable String cacheName) { return !staleFuture(fut0) && (cacheName == null || opCtx.containsCache(cacheName)); } - /** - * @return Interrupted flag. - */ - private boolean interrupted() { - return errRef.get() != null; - } - - /** - * Interrupt process. - * - * @param err Error. - * @return {@code True} if process has been interrupted by this call. - */ - private boolean interrupt(Exception err) { - return errRef.compareAndSet(null, err); - } - /** * @param fut The future of cache snapshot restore operation. * @return {@code True} if the future completed or not initiated. @@ -226,11 +205,10 @@ public boolean staleFuture(IgniteInternalFuture fut) { * @param leftNodeId Left node ID. */ public void onNodeLeft(UUID leftNodeId) { - if (staleFuture(fut)) - return; + SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx.nodes().contains(leftNodeId)) { - interrupt(new IgniteException(OP_REJECT_MSG + + if (opCtx0 != null && opCtx0.nodes().contains(leftNodeId)) { + opCtx0.interrupt(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -241,12 +219,10 @@ public void onNodeLeft(UUID leftNodeId) { * @param reason Interruption reason. */ public void stop(Exception reason) { - IgniteInternalFuture fut0 = fut; - - if (staleFuture(fut0)) - return; + SnapshotRestoreContext opCtx0 = opCtx; - interrupt(reason); + if (opCtx0 != null) + opCtx0.interrupt(reason); } /** @@ -286,10 +262,7 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), new HashSet<>(req.nodes()), req.configs(), ctx); - fut.listen(f -> { - opCtx = null; - errRef.set(null); - }); + fut.listen(f -> opCtx = null); if (!baselineNodes().containsAll(req.nodes())) { return new GridFinishedFuture<>( @@ -316,15 +289,15 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq ctx.getSystemExecutorService().submit(() -> { try { - opCtx0.restore(updateMeta, this::interrupted); + opCtx0.restore(updateMeta, opCtx0::interrupted); - if (interrupted()) { + if (opCtx0.interrupted()) { log.error("Snapshot restore process has been interrupted " + - "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', errRef.get()); + "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', opCtx0.error()); opCtx0.rollback(); - retFut.onDone(errRef.get()); + retFut.onDone(opCtx0.error()); } else retFut.onDone(); @@ -382,10 +355,10 @@ private IgniteInternalFuture cacheStart(SnapshotRestore return new GridFinishedFuture<>(); if (ctx.state().clusterState().state() != ClusterState.ACTIVE) - return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Cluster state has been changed.")); + return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active.")); - if (interrupted()) - return new GridFinishedFuture<>(errRef.get()); + if (opCtx0.interrupted()) + return new GridFinishedFuture<>(opCtx0.error()); if (!baselineNodes().containsAll(opCtx0.nodes())) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index 2c1212c2c4f1c5..a068ba6dd1b26a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -28,6 +28,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; @@ -35,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; @@ -66,6 +68,9 @@ class SnapshotRestoreContext { /** Restored cache groups. */ private final Map> grps = new ConcurrentHashMap<>(); + /** The exception that led to the interruption of the process. */ + private final AtomicReference errRef = new AtomicReference<>(); + /** * @param reqId Request ID. * @param snpName Snapshot name. @@ -73,7 +78,7 @@ class SnapshotRestoreContext { * @param configs Stored cache configurations. * @param ctx Kernal context. */ - public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List configs, + protected SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List configs, GridKernalContext ctx) { ccfgs = new ArrayList<>(configs); @@ -97,24 +102,24 @@ public SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, Li } /** @return Request ID. */ - public UUID requestId() { + protected UUID requestId() { return reqId; } /** @return List of baseline node IDs that must be alive to complete the operation. */ - public Set nodes() { + protected Set nodes() { return Collections.unmodifiableSet(reqNodes); } /** @return Snapshot name. */ - public String snapshotName() { + protected String snapshotName() { return snpName; } /** * @return List of cache group names to restore from the snapshot. */ - public Set groups() { + protected Set groups() { return grps.keySet(); } @@ -122,15 +127,39 @@ public Set groups() { * @param name Cache name. * @return {@code True} if the cache with the specified name is currently being restored. */ - public boolean containsCache(String name) { + protected boolean containsCache(String name) { return cacheIds.contains(CU.cacheId(name)); } /** @return Cache configurations. */ - public Collection configs() { + protected Collection configs() { return ccfgs; } + /** + * @return Interrupted flag. + */ + protected boolean interrupted() { + return error() != null; + } + + /** + * @return Error or {@code null} if there were no errors. + */ + protected @Nullable Throwable error() { + return errRef.get(); + } + + /** + * Interrupt process. + * + * @param err Error. + * @return {@code True} if process has been interrupted by this call. + */ + protected boolean interrupt(Exception err) { + return errRef.compareAndSet(null, err); + } + /** * Restore specified cache groups from the local snapshot directory. * @@ -138,7 +167,7 @@ public Collection configs() { * @param stopChecker Node stop or prcoess interrupt checker. * @throws IgniteCheckedException If failed. */ - public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws IgniteCheckedException { + protected void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws IgniteCheckedException { if (stopChecker.getAsBoolean()) return; @@ -169,7 +198,7 @@ public void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws /** * Rollback changes made by process in specified cache group. */ - public void rollback() { + protected void rollback() { rollbackLock.lock(); try { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java index 67280ef6b9f902..730b9e131c6db7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/util/distributed/DistributedProcess.java @@ -460,6 +460,6 @@ public enum DistributedProcessType { /** * Cache group restore rollback phase. */ - RESTORE_CACHE_GROUP_SNAPSHOT_FINISH + RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK } } 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 13243f8a5b44e7..986ffe012784d9 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 @@ -55,13 +55,11 @@ import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.Nullable; -import org.junit.Ignore; import org.junit.Test; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_FINISH; 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.runAsync; @@ -194,47 +192,6 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); } - /** @throws Exception If failed. */ - @Test - @Ignore - public void testRestoreWithMissedPartitions() throws Exception { - IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, dfltCacheCfg.setBackups(0)); - - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - - ignite.cache(dfltCacheCfg.getName()).destroy(); - - awaitPartitionMapExchange(); - - forceCheckpoint(); - - stopGrid(1); - - IgniteFuture fut = - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); - - GridTestUtils.assertThrowsAnyCause( - log, () -> fut.get(TIMEOUT), IgniteException.class, "not all partitions available"); - - startGrid(1); - - IgniteFuture fut1 = - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); - - GridTestUtils.assertThrowsAnyCause( - log, () -> fut1.get(TIMEOUT), IllegalStateException.class, - "Cache \"" + dfltCacheCfg.getName() + "\" should be destroyed manually"); - - ignite.cache(dfltCacheCfg.getName()).destroy(); - - awaitPartitionMapExchange(); - - ignite.snapshot().restoreCacheGroups( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); - } - /** @throws Exception If failed. */ @Test public void testClusterSnapshotRestoreDiffTopology() throws Exception { @@ -595,7 +552,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { public void testClusterStateChangeActiveReadonlyOnPrepare() throws Exception { checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, IgniteCheckedException.class, - "Cluster state has been changed"); + "The cluster should be active."); } /** @@ -606,21 +563,13 @@ public void testClusterStateChangeActiveReadonlyOnCacheStart() throws Exception checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_START, null, null); } -// /** -// * @throws Exception if failed. -// */ -// @Test -// public void testClusterStateChangeActiveReadonlyOnFinish() throws Exception { -// checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, null, null); -// } - /** * @throws Exception if failed. */ @Test public void testClusterDeactivateOnPrepare() throws Exception { checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - IgniteCheckedException.class, "Cluster state has been changed."); + IgniteCheckedException.class, "The cluster should be active."); } /** @@ -631,16 +580,6 @@ public void testClusterDeactivateOnCacheStart() throws Exception { checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_START, null, null); } - /** - * @throws Exception if failed. - */ - @Test - @Ignore - public void testClusterDeactivateOnFinish() throws Exception { - checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_FINISH, - IgniteException.class, "Server node(s) has left the cluster", true); - } - /** * @param state Cluster state. * @param procType The type of distributed process on which communication is blocked. From 2b8891dd8f8fb479a6f5aa1b36fba04476418322 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 19 Feb 2021 08:56:52 +0300 Subject: [PATCH 29/98] IGNITE-13805 Code cleanup. --- .../SnapshotRestoreCacheGroupProcess.java | 4 +- .../snapshot/SnapshotRestoreContext.java | 74 +++++++------------ .../snapshot/SnapshotRestoreRequest.java | 8 +- .../SnapshotRestoreVerificatioTask.java | 2 +- 4 files changed, 35 insertions(+), 53 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index a6fd95489f884f..8ddeba82392ef4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -260,7 +260,7 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq if (fut.isDone()) fut = new GridFutureAdapter<>(); - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), new HashSet<>(req.nodes()), req.configs(), ctx); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.configs(), ctx); fut.listen(f -> opCtx = null); @@ -289,7 +289,7 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq ctx.getSystemExecutorService().submit(() -> { try { - opCtx0.restore(updateMeta, opCtx0::interrupted); + opCtx0.restore(updateMeta); if (opCtx0.interrupted()) { log.error("Snapshot restore process has been interrupted " + diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java index a068ba6dd1b26a..a8ccf2af2bfba1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java @@ -29,8 +29,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; -import java.util.concurrent.locks.ReentrantLock; -import java.util.function.BooleanSupplier; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; @@ -50,15 +48,12 @@ class SnapshotRestoreContext { /** Snapshot name. */ private final String snpName; - /** List of baseline node IDs that must be alive to complete the operation. */ + /** Baseline node IDs that must be alive to complete the operation. */ private final Set reqNodes; /** Kernal context. */ private final GridKernalContext ctx; - /** Restore operation lock. */ - private final ReentrantLock rollbackLock = new ReentrantLock(); - /** List of processed cache IDs. */ private final Set cacheIds = new HashSet<>(); @@ -74,7 +69,7 @@ class SnapshotRestoreContext { /** * @param reqId Request ID. * @param snpName Snapshot name. - * @param reqNodes List of baseline node IDs that must be alive to complete the operation. + * @param reqNodes Baseline node IDs that must be alive to complete the operation. * @param configs Stored cache configurations. * @param ctx Kernal context. */ @@ -96,7 +91,7 @@ protected SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, } this.reqId = reqId; - this.reqNodes = reqNodes; + this.reqNodes = new HashSet<>(reqNodes); this.snpName = snpName; this.ctx = ctx; } @@ -106,7 +101,7 @@ protected UUID requestId() { return reqId; } - /** @return List of baseline node IDs that must be alive to complete the operation. */ + /** @return Baseline node IDs that must be alive to complete the operation. */ protected Set nodes() { return Collections.unmodifiableSet(reqNodes); } @@ -136,6 +131,14 @@ protected Collection configs() { return ccfgs; } + /** + * @param err Error. + * @return {@code True} if operation has been interrupted by this call. + */ + protected boolean interrupt(Exception err) { + return errRef.compareAndSet(null, err); + } + /** * @return Interrupted flag. */ @@ -144,35 +147,26 @@ protected boolean interrupted() { } /** - * @return Error or {@code null} if there were no errors. + * @return Error if operation was interrupted, otherwise {@code null}. */ protected @Nullable Throwable error() { return errRef.get(); } - /** - * Interrupt process. - * - * @param err Error. - * @return {@code True} if process has been interrupted by this call. - */ - protected boolean interrupt(Exception err) { - return errRef.compareAndSet(null, err); - } - /** * Restore specified cache groups from the local snapshot directory. * * @param updateMetadata Update binary metadata flag. - * @param stopChecker Node stop or prcoess interrupt checker. * @throws IgniteCheckedException If failed. */ - protected void restore(boolean updateMetadata, BooleanSupplier stopChecker) throws IgniteCheckedException { - if (stopChecker.getAsBoolean()) + protected void restore(boolean updateMetadata) throws IgniteCheckedException { + if (interrupted()) return; + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + if (updateMetadata) { - File binDir = binaryWorkDir(ctx.cache().context().snapshotMgr().snapshotLocalDir(snpName).getAbsolutePath(), + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), ctx.pdsFolderResolver().resolveFolders().folderName()); if (!binDir.exists()) { @@ -180,39 +174,27 @@ protected void restore(boolean updateMetadata, BooleanSupplier stopChecker) thro "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); } - ctx.cacheObjects().updateMetadata(binDir, stopChecker); + ctx.cacheObjects().updateMetadata(binDir, this::interrupted); } - for (String grpName : groups()) { - rollbackLock.lock(); - - try { - ctx.cache().context().snapshotMgr().restoreCacheGroupFiles(snpName, grpName, stopChecker, grps.get(grpName)); - } - finally { - rollbackLock.unlock(); - } - } + for (String grpName : groups()) + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, this::interrupted, grps.get(grpName)); } /** * Rollback changes made by process in specified cache group. */ protected void rollback() { - rollbackLock.lock(); + if (groups().isEmpty()) + return; - try { - List grpNames = new ArrayList<>(groups()); + List grpNames = new ArrayList<>(groups()); - for (String grpName : grpNames) { - List files = grps.remove(grpName); + for (String grpName : grpNames) { + List files = grps.remove(grpName); - if (files != null) - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); - } - } - finally { - rollbackLock.unlock(); + if (files != null) + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java index e4471b44fa6ce0..cedec1a28527f2 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java @@ -18,8 +18,8 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import java.io.Serializable; -import java.util.Collection; import java.util.List; +import java.util.Set; import java.util.UUID; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.tostring.GridToStringExclude; @@ -39,7 +39,7 @@ public class SnapshotRestoreRequest implements Serializable { private final String snpName; /** Baseline node IDs that must be alive to complete the operation. */ - private final Collection nodes; + private final Set nodes; /** Stored cache configurations. */ @GridToStringExclude @@ -58,7 +58,7 @@ public class SnapshotRestoreRequest implements Serializable { public SnapshotRestoreRequest( UUID reqId, String snpName, - Collection nodes, + Set nodes, List ccfgs, UUID updateMetaNodeId ) { @@ -93,7 +93,7 @@ public List configs() { /** * @return Baseline node IDs that must be alive to complete the operation. */ - public Collection nodes() { + public Set nodes() { return nodes; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java index 5822ef1819772b..f878f061fb56f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java @@ -62,7 +62,7 @@ public class SnapshotRestoreVerificatioTask extends return jobs; } - + SnapshotRestoreRequest /** {@inheritDoc} */ @Override public SnapshotRestoreVerificationResult reduce(List results) throws IgniteException { SnapshotRestoreVerificationResult firstRes = null; From f2785a7d9015cce52869bb5e0d7e208d40e35c93 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 1 Mar 2021 11:53:13 +0300 Subject: [PATCH 30/98] IGNITE-13805 Code cleanup. --- .../ignite/internal/IgniteFeatures.java | 5 +- .../SnapshotRestoreCacheGroupProcess.java | 98 ++++++++++++------- .../SnapshotRestoreCacheStartRequest.java | 52 ++++++++++ ...java => SnapshotRestoreEmptyResponse.java} | 2 +- ...ava => SnapshotRestorePrepareRequest.java} | 6 +- .../SnapshotRestoreRollbackRequest.java | 2 +- .../SnapshotRestoreVerificatioTask.java | 2 +- .../IgniteClusterSnapshotRestoreSelfTest.java | 5 +- 8 files changed, 124 insertions(+), 48 deletions(-) create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestoreResponse.java => SnapshotRestoreEmptyResponse.java} (93%) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestoreRequest.java => SnapshotRestorePrepareRequest.java} (94%) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index fa487a0554dfa2..4f8fdba44cbd1e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -139,7 +139,10 @@ public enum IgniteFeatures { CACHE_GROUP_KEY_CHANGE(47), /** Collecting performance statistics. */ - PERFORMANCE_STATISTICS(48); + PERFORMANCE_STATISTICS(48), + + /** Restore cache group from the snapshot. */ + SNAPSHOT_RESTORE_CACHE_GROUP(49); /** * Unique feature identifier. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 8ddeba82392ef4..637ff7f6dce548 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -25,11 +25,12 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.internal.GridKernalContext; +import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; @@ -46,6 +47,7 @@ import org.apache.ignite.lang.IgniteFuture; import org.jetbrains.annotations.Nullable; +import static org.apache.ignite.internal.IgniteFeatures.SNAPSHOT_RESTORE_CACHE_GROUP; 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_ROLLBACK; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_START; @@ -61,10 +63,10 @@ public class SnapshotRestoreCacheGroupProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess prepareRestoreProc; + private final DistributedProcess prepareRestoreProc; /** Cache group restore cache start phase. */ - private final DistributedProcess cacheStartProc; + private final DistributedProcess cacheStartProc; /** Cache group restore rollback phase. */ private final DistributedProcess rollbackRestoreProc; @@ -120,7 +122,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames DiscoveryDataClusterState clusterState = ctx.state().clusterState(); - if (ctx.state().clusterState().state() != ClusterState.ACTIVE) + DiscoveryDataClusterState state = ctx.state().clusterState(); + + if (state.state() != ClusterState.ACTIVE || state.transition()) return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); if (!clusterState.hasBaselineTopology()) { @@ -133,7 +137,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "A cluster snapshot operation is in progress.")); } - Set bltNodeIds = baselineNodes(); + Collection bltNodes = F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + node -> node, (node) -> CU.baselineNode(node, ctx.state().clusterState())); + + if (!IgniteFeatures.allNodesSupports(bltNodes, SNAPSHOT_RESTORE_CACHE_GROUP)) + throw new IgniteException("Not all nodes in the cluster support a snapshot restore operation."); + + Set bltNodeIds = new HashSet<>(F.viewReadOnly(bltNodes, F.node2id())); fut = new GridFutureAdapter<>(); @@ -159,7 +169,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - SnapshotRestoreRequest req = new SnapshotRestoreRequest( + SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest( UUID.randomUUID(), snpName, bltNodeIds, res.configs(), res.localNodeId()); prepareRestoreProc.start(req.requestId(), req); @@ -172,14 +182,6 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return new IgniteFutureImpl<>(fut); } - /** - * @return Set of current baseline node IDs. - */ - private Set baselineNodes() { - return new HashSet<>(F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - F.node2id(), (node) -> CU.baselineNode(node, ctx.state().clusterState()))); - } - /** * Check if the cache group restore process is currently running. * @@ -244,7 +246,7 @@ private void ensureCacheAbsent(String name) throws IllegalStateException { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture prepare(SnapshotRestoreRequest req) { + private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { if (!req.nodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); @@ -253,8 +255,10 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); } - if (ctx.state().clusterState().state() != ClusterState.ACTIVE) - return new GridFinishedFuture<>(new IllegalStateException(OP_REJECT_MSG + "The cluster should be active.")); + DiscoveryDataClusterState state = ctx.state().clusterState(); + + if (state.state() != ClusterState.ACTIVE || state.transition()) + return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); // Skip creating future on initiator. if (fut.isDone()) @@ -264,14 +268,12 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq fut.listen(f -> opCtx = null); - if (!baselineNodes().containsAll(req.nodes())) { - return new GridFinishedFuture<>( - new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); - } + if (!allNodesInBaselineAndAlive(req.nodes())) + return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); SnapshotRestoreContext opCtx0 = opCtx; - GridFutureAdapter retFut = new GridFutureAdapter<>(); + GridFutureAdapter retFut = new GridFutureAdapter<>(); try { for (String grpName : opCtx0.groups()) @@ -291,16 +293,19 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq try { opCtx0.restore(updateMeta); - if (opCtx0.interrupted()) { - log.error("Snapshot restore process has been interrupted " + - "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', opCtx0.error()); - - opCtx0.rollback(); + if (!opCtx0.interrupted()) { + retFut.onDone(); - retFut.onDone(opCtx0.error()); + return; } - else - retFut.onDone(); + + log.error("Snapshot restore process has been interrupted " + + "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', opCtx0.error()); + + opCtx0.rollback(); + + retFut.onDone(opCtx0.error()); + } catch (Throwable t) { retFut.onDone(t); @@ -318,7 +323,7 @@ private IgniteInternalFuture prepare(SnapshotRestoreReq * @param res Results. * @param errs Errors. */ - private void finishPrepare(UUID reqId, Map res, Map errs) { + private void finishPrepare(UUID reqId, Map res, Map errs) { GridFutureAdapter fut0 = fut; if (fut0.isDone() || !reqId.equals(opCtx.requestId())) @@ -335,14 +340,14 @@ private void finishPrepare(UUID reqId, Map res, M } if (U.isLocalNodeCoordinator(ctx.discovery())) - cacheStartProc.start(reqId, new SnapshotRestoreRequest(reqId, null, null, null, null)); + cacheStartProc.start(reqId, new SnapshotRestoreCacheStartRequest(reqId)); } /** * @param req Request to start restored cache groups. * @return Result future. */ - private IgniteInternalFuture cacheStart(SnapshotRestoreRequest req) { + private IgniteInternalFuture cacheStart(SnapshotRestoreCacheStartRequest req) { SnapshotRestoreContext opCtx0 = opCtx; if (staleFuture(fut)) @@ -354,16 +359,18 @@ private IgniteInternalFuture cacheStart(SnapshotRestore if (!U.isLocalNodeCoordinator(ctx.discovery())) return new GridFinishedFuture<>(); - if (ctx.state().clusterState().state() != ClusterState.ACTIVE) - return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active.")); + DiscoveryDataClusterState state = ctx.state().clusterState(); + + if (state.state() != ClusterState.ACTIVE || state.transition()) + return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); if (opCtx0.interrupted()) return new GridFinishedFuture<>(opCtx0.error()); - if (!baselineNodes().containsAll(opCtx0.nodes())) + if (!allNodesInBaselineAndAlive(opCtx0.nodes())) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); - GridFutureAdapter retFut = new GridFutureAdapter<>(); + GridFutureAdapter retFut = new GridFutureAdapter<>(); if (log.isInfoEnabled()) { log.info("Starting restored caches " + @@ -391,7 +398,7 @@ private IgniteInternalFuture cacheStart(SnapshotRestore * @param res Results. * @param errs Errors. */ - private void finishCacheStart(UUID reqId, Map res, Map errs) { + private void finishCacheStart(UUID reqId, Map res, Map errs) { GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; @@ -454,4 +461,19 @@ private void finishRollback(UUID reqId, Map nodeIds) { + for (UUID nodeId : nodeIds) { + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null || !CU.baselineNode(node, ctx.state().clusterState()) || !ctx.discovery().alive(node)) + return false; + } + + return true; + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java new file mode 100644 index 00000000000000..8e1ed2d806e505 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java @@ -0,0 +1,52 @@ +/* + * 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.UUID; +import org.apache.ignite.internal.util.typedef.internal.S; + +/** + * Request to start restored cache group. + */ +public class SnapshotRestoreCacheStartRequest implements Serializable { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Request ID. */ + private final UUID reqId; + + /** + * @param reqId Request ID. + */ + public SnapshotRestoreCacheStartRequest(UUID reqId) { + this.reqId = reqId; + } + + /** + * @return Request ID. + */ + public UUID requestId() { + return reqId; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(SnapshotRestoreCacheStartRequest.class, this); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java similarity index 93% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java index 79601b00f5b581..0cd4bb431739b6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreResponse.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java @@ -22,7 +22,7 @@ /** * Snapshot restore operation single node response. */ -public class SnapshotRestoreResponse implements Serializable { +public class SnapshotRestoreEmptyResponse implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java similarity index 94% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index cedec1a28527f2..4ffd104f5efae6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -28,7 +28,7 @@ /** * Request to prepare cache group restore from the snapshot. */ -public class SnapshotRestoreRequest implements Serializable { +public class SnapshotRestorePrepareRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -55,7 +55,7 @@ public class SnapshotRestoreRequest implements Serializable { * @param ccfgs Stored cache configurations. * @param updateMetaNodeId Node ID from which to update the binary metadata. */ - public SnapshotRestoreRequest( + public SnapshotRestorePrepareRequest( UUID reqId, String snpName, Set nodes, @@ -106,6 +106,6 @@ public UUID updateMetaNodeId() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SnapshotRestoreRequest.class, this); + return S.toString(SnapshotRestorePrepareRequest.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java index a5089aa09bae05..87e9d74e0d1674 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java @@ -39,7 +39,7 @@ public class SnapshotRestoreRollbackRequest implements Serializable { * @param reqId Request ID. * @param err Process execution error. */ - public SnapshotRestoreRollbackRequest(UUID reqId, @Nullable Throwable err) { + public SnapshotRestoreRollbackRequest(UUID reqId, Throwable err) { this.reqId = reqId; this.err = err; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java index f878f061fb56f8..5822ef1819772b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java @@ -62,7 +62,7 @@ public class SnapshotRestoreVerificatioTask extends return jobs; } - SnapshotRestoreRequest + /** {@inheritDoc} */ @Override public SnapshotRestoreVerificationResult reduce(List results) throws IgniteException { SnapshotRestoreVerificationResult firstRes = null; 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 986ffe012784d9..7755516a786e64 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 @@ -551,8 +551,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { @Test public void testClusterStateChangeActiveReadonlyOnPrepare() throws Exception { checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - IgniteCheckedException.class, - "The cluster should be active."); + IgniteException.class, "The cluster should be active."); } /** @@ -569,7 +568,7 @@ public void testClusterStateChangeActiveReadonlyOnCacheStart() throws Exception @Test public void testClusterDeactivateOnPrepare() throws Exception { checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - IgniteCheckedException.class, "The cluster should be active."); + IgniteException.class, "The cluster should be active."); } /** From d9bdc844992f61273f2c4a69bfe96054ff486512 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 2 Mar 2021 11:36:42 +0300 Subject: [PATCH 31/98] IGNITE-13805 Moved restore context to inner class. --- .../SnapshotRestoreCacheGroupProcess.java | 164 +++++++++++++- .../snapshot/SnapshotRestoreContext.java | 205 ------------------ 2 files changed, 163 insertions(+), 206 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 637ff7f6dce548..5334ecba7ee88e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -17,14 +17,20 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; +import java.io.File; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; @@ -48,6 +54,7 @@ import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.IgniteFeatures.SNAPSHOT_RESTORE_CACHE_GROUP; +import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; 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_ROLLBACK; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_START; @@ -264,7 +271,7 @@ private IgniteInternalFuture prepare(SnapshotResto if (fut.isDone()) fut = new GridFutureAdapter<>(); - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.configs(), ctx); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.configs()); fut.listen(f -> opCtx = null); @@ -476,4 +483,159 @@ private boolean allNodesInBaselineAndAlive(Set nodeIds) { return true; } + + /** + * Cache group restore from snapshot operation context. + */ + private class SnapshotRestoreContext { + /** Request ID. */ + private final UUID reqId; + + /** Snapshot name. */ + private final String snpName; + + /** Baseline node IDs that must be alive to complete the operation. */ + private final Set reqNodes; + + /** List of processed cache IDs. */ + private final Set cacheIds = new HashSet<>(); + + /** Cache configurations. */ + private final List ccfgs; + + /** Restored cache groups. */ + private final Map> grps = new ConcurrentHashMap<>(); + + /** The exception that led to the interruption of the process. */ + private final AtomicReference errRef = new AtomicReference<>(); + + /** + * @param reqId Request ID. + * @param snpName Snapshot name. + * @param reqNodes Baseline node IDs that must be alive to complete the operation. + * @param cfgs Stored cache configurations. + */ + protected SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List cfgs) { + ccfgs = new ArrayList<>(cfgs); + + for (StoredCacheData cacheData : cfgs) { + String cacheName = cacheData.config().getName(); + + cacheIds.add(CU.cacheId(cacheName)); + + boolean shared = cacheData.config().getGroupName() != null; + + grps.computeIfAbsent(shared ? cacheData.config().getGroupName() : cacheName, v -> new ArrayList<>()); + + if (shared) + cacheIds.add(CU.cacheId(cacheData.config().getGroupName())); + } + + this.reqId = reqId; + this.reqNodes = new HashSet<>(reqNodes); + this.snpName = snpName; + } + + /** @return Request ID. */ + protected UUID requestId() { + return reqId; + } + + /** @return Baseline node IDs that must be alive to complete the operation. */ + protected Set nodes() { + return Collections.unmodifiableSet(reqNodes); + } + + /** @return Snapshot name. */ + protected String snapshotName() { + return snpName; + } + + /** + * @return List of cache group names to restore from the snapshot. + */ + protected Set groups() { + return grps.keySet(); + } + + /** + * @param name Cache name. + * @return {@code True} if the cache with the specified name is currently being restored. + */ + protected boolean containsCache(String name) { + return cacheIds.contains(CU.cacheId(name)); + } + + /** @return Cache configurations. */ + protected Collection configs() { + return ccfgs; + } + + /** + * @param err Error. + * @return {@code True} if operation has been interrupted by this call. + */ + protected boolean interrupt(Exception err) { + return errRef.compareAndSet(null, err); + } + + /** + * @return Interrupted flag. + */ + protected boolean interrupted() { + return error() != null; + } + + /** + * @return Error if operation was interrupted, otherwise {@code null}. + */ + protected @Nullable Throwable error() { + return errRef.get(); + } + + /** + * Restore specified cache groups from the local snapshot directory. + * + * @param updateMetadata Update binary metadata flag. + * @throws IgniteCheckedException If failed. + */ + protected void restore(boolean updateMetadata) throws IgniteCheckedException { + if (interrupted()) + return; + + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + + if (updateMetadata) { + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), + ctx.pdsFolderResolver().resolveFolders().folderName()); + + if (!binDir.exists()) { + throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + + "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); + } + + ctx.cacheObjects().updateMetadata(binDir, this::interrupted); + } + + for (String grpName : groups()) + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, this::interrupted, grps.get(grpName)); + } + + /** + * Rollback changes made by process in specified cache group. + */ + protected void rollback() { + if (groups().isEmpty()) + return; + + List grpNames = new ArrayList<>(groups()); + + for (String grpName : grpNames) { + List files = grps.remove(grpName); + + if (files != null) + ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); + } + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java deleted file mode 100644 index a8ccf2af2bfba1..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreContext.java +++ /dev/null @@ -1,205 +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.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.ignite.IgniteCheckedException; -import org.apache.ignite.internal.GridKernalContext; -import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.S; -import org.jetbrains.annotations.Nullable; - -import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; - -/** - * Cache group restore from snapshot operation context. - */ -class SnapshotRestoreContext { - /** Request ID. */ - private final UUID reqId; - - /** Snapshot name. */ - private final String snpName; - - /** Baseline node IDs that must be alive to complete the operation. */ - private final Set reqNodes; - - /** Kernal context. */ - private final GridKernalContext ctx; - - /** List of processed cache IDs. */ - private final Set cacheIds = new HashSet<>(); - - /** Cache configurations. */ - private final List ccfgs; - - /** Restored cache groups. */ - private final Map> grps = new ConcurrentHashMap<>(); - - /** The exception that led to the interruption of the process. */ - private final AtomicReference errRef = new AtomicReference<>(); - - /** - * @param reqId Request ID. - * @param snpName Snapshot name. - * @param reqNodes Baseline node IDs that must be alive to complete the operation. - * @param configs Stored cache configurations. - * @param ctx Kernal context. - */ - protected SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List configs, - GridKernalContext ctx) { - ccfgs = new ArrayList<>(configs); - - for (StoredCacheData cacheData : configs) { - String cacheName = cacheData.config().getName(); - - cacheIds.add(CU.cacheId(cacheName)); - - boolean shared = cacheData.config().getGroupName() != null; - - grps.computeIfAbsent(shared ? cacheData.config().getGroupName() : cacheName, v -> new ArrayList<>()); - - if (shared) - cacheIds.add(CU.cacheId(cacheData.config().getGroupName())); - } - - this.reqId = reqId; - this.reqNodes = new HashSet<>(reqNodes); - this.snpName = snpName; - this.ctx = ctx; - } - - /** @return Request ID. */ - protected UUID requestId() { - return reqId; - } - - /** @return Baseline node IDs that must be alive to complete the operation. */ - protected Set nodes() { - return Collections.unmodifiableSet(reqNodes); - } - - /** @return Snapshot name. */ - protected String snapshotName() { - return snpName; - } - - /** - * @return List of cache group names to restore from the snapshot. - */ - protected Set groups() { - return grps.keySet(); - } - - /** - * @param name Cache name. - * @return {@code True} if the cache with the specified name is currently being restored. - */ - protected boolean containsCache(String name) { - return cacheIds.contains(CU.cacheId(name)); - } - - /** @return Cache configurations. */ - protected Collection configs() { - return ccfgs; - } - - /** - * @param err Error. - * @return {@code True} if operation has been interrupted by this call. - */ - protected boolean interrupt(Exception err) { - return errRef.compareAndSet(null, err); - } - - /** - * @return Interrupted flag. - */ - protected boolean interrupted() { - return error() != null; - } - - /** - * @return Error if operation was interrupted, otherwise {@code null}. - */ - protected @Nullable Throwable error() { - return errRef.get(); - } - - /** - * Restore specified cache groups from the local snapshot directory. - * - * @param updateMetadata Update binary metadata flag. - * @throws IgniteCheckedException If failed. - */ - protected void restore(boolean updateMetadata) throws IgniteCheckedException { - if (interrupted()) - return; - - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - - if (updateMetadata) { - File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), - ctx.pdsFolderResolver().resolveFolders().folderName()); - - if (!binDir.exists()) { - throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + - "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); - } - - ctx.cacheObjects().updateMetadata(binDir, this::interrupted); - } - - for (String grpName : groups()) - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, this::interrupted, grps.get(grpName)); - } - - /** - * Rollback changes made by process in specified cache group. - */ - protected void rollback() { - if (groups().isEmpty()) - return; - - List grpNames = new ArrayList<>(groups()); - - for (String grpName : grpNames) { - List files = grps.remove(grpName); - - if (files != null) - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); - } - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotRestoreContext.class, this); - } -} From 841740f225f5573f9f71518f491745a9484ed0be Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 4 Mar 2021 09:44:12 +0300 Subject: [PATCH 32/98] IGNITE-13805 (minor) Code cleanup. --- .../snapshot/IgniteSnapshotManager.java | 7 ++-- .../SnapshotRestoreCacheGroupProcess.java | 35 +++++++++---------- .../SnapshotRestoreVerificatioTask.java | 7 ++-- 3 files changed, 23 insertions(+), 26 deletions(-) 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 8130b9b74a27ff..cd04ac6a109e72 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 @@ -1125,9 +1125,9 @@ protected void restoreCacheGroupFiles( File.separator + pdsSettings.folderName() + File.separator + snapshotCacheDir.getName(), false); if (!cacheDir.exists()) { - cacheDir.mkdir(); - newFiles.add(cacheDir); + + cacheDir.mkdir(); } else if (cacheDir.list().length > 0) { @@ -1191,8 +1191,7 @@ protected void rollbackRestoreOperation(Collection files) { * @return Local path to the cache directory. */ public File resolveSnapshotCacheDir(String snpName, String cacheName) { - File dbDir = Paths.get(snapshotLocalDir(snpName).getAbsolutePath(), - DFLT_STORE_DIR, pdsSettings.folderName()).toFile(); + File dbDir = new File(snapshotLocalDir(snpName), databaseRelativePath(pdsSettings.folderName())); File cacheDir = new File(dbDir, CACHE_DIR_PREFIX + cacheName); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index 5334ecba7ee88e..e621dea2f54fa0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -129,9 +129,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames DiscoveryDataClusterState clusterState = ctx.state().clusterState(); - DiscoveryDataClusterState state = ctx.state().clusterState(); - - if (state.state() != ClusterState.ACTIVE || state.transition()) + if (clusterState.state() != ClusterState.ACTIVE || clusterState.transition()) return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); if (!clusterState.hasBaselineTopology()) { @@ -144,12 +142,12 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "A cluster snapshot operation is in progress.")); } + if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) + throw new IgniteException("Not all nodes in the cluster support a snapshot restore operation."); + Collection bltNodes = F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), node -> node, (node) -> CU.baselineNode(node, ctx.state().clusterState())); - if (!IgniteFeatures.allNodesSupports(bltNodes, SNAPSHOT_RESTORE_CACHE_GROUP)) - throw new IgniteException("Not all nodes in the cluster support a snapshot restore operation."); - Set bltNodeIds = new HashSet<>(F.viewReadOnly(bltNodes, F.node2id())); fut = new GridFutureAdapter<>(); @@ -169,9 +167,8 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames missedGroups.removeAll(foundGrps); - fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + - "Cache group(s) was not found in the snapshot [groups=" + - F.concat(missedGroups, ", ") + ", snapshot=" + snpName + ']')); + fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + "snapshot [groups=" + missedGroups + ", snapshot=" + snpName + ']')); return; } @@ -195,16 +192,17 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * @return {@code True} if cache group restore process is currently running. */ public boolean inProgress(@Nullable String cacheName) { - IgniteInternalFuture fut0 = fut; + SnapshotRestoreContext opCtx0 = opCtx; - return !staleFuture(fut0) && (cacheName == null || opCtx.containsCache(cacheName)); + return !staleProcess(fut, opCtx0) && (cacheName == null || opCtx0.containsCache(cacheName)); } /** * @param fut The future of cache snapshot restore operation. + * @param opCtx Snapshot restore operation context. * @return {@code True} if the future completed or not initiated. */ - public boolean staleFuture(IgniteInternalFuture fut) { + public boolean staleProcess(IgniteInternalFuture fut, SnapshotRestoreContext opCtx) { return fut.isDone() || opCtx == null; } @@ -357,7 +355,7 @@ private void finishPrepare(UUID reqId, Map r private IgniteInternalFuture cacheStart(SnapshotRestoreCacheStartRequest req) { SnapshotRestoreContext opCtx0 = opCtx; - if (staleFuture(fut)) + if (staleProcess(fut, opCtx0)) return new GridFinishedFuture<>(); if (!req.requestId().equals(opCtx0.requestId())) @@ -409,7 +407,7 @@ private void finishCacheStart(UUID reqId, Map fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; - if (staleFuture(fut0) || !reqId.equals(opCtx0.requestId())) + if (staleProcess(fut0, opCtx0) || !reqId.equals(opCtx0.requestId())) return; Exception failure = F.first(errs.values()); @@ -437,11 +435,11 @@ private void finishCacheStart(UUID reqId, Map rollback(SnapshotRestoreRollbackRequest req) { - if (staleFuture(fut) || !req.requestId().equals(opCtx.requestId())) - return new GridFinishedFuture<>(); - SnapshotRestoreContext opCtx0 = opCtx; + if (staleProcess(fut, opCtx0) || !req.requestId().equals(opCtx0.requestId())) + return new GridFinishedFuture<>(); + if (!opCtx0.nodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); @@ -460,8 +458,9 @@ private IgniteInternalFuture rollback(SnapshotR */ private void finishRollback(UUID reqId, Map res, Map errs) { GridFutureAdapter fut0 = fut; + SnapshotRestoreContext opCtx0 = opCtx; - if (staleFuture(fut0) || !reqId.equals(opCtx.requestId())) + if (staleProcess(fut0, opCtx0) || !reqId.equals(opCtx0.requestId())) return; SnapshotRestoreRollbackResponse resp = F.first(F.viewReadOnly(res.values(), v -> v, Objects::nonNull)); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java index 5822ef1819772b..8390e5396e2a53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java @@ -42,7 +42,6 @@ import org.jetbrains.annotations.NotNull; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; /** * Verification task for restoring a cache group from a snapshot. @@ -153,13 +152,13 @@ private SnapshotRestoreVerificationResult resolveRestoredConfigs() throws Ignite ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cacheCfgs); - File[] parts = cacheDir.listFiles(f -> f.getName().startsWith(PART_FILE_PREFIX) && !f.isDirectory()); + List parts = FilePageStoreManager.cachePartitionFiles(cacheDir); if (F.isEmpty(parts)) continue; - int pageSize = - ((GridCacheDatabaseSharedManager)cctx.database()).resolvePageSizeFromPartitionFile(parts[0].toPath()); + int pageSize = ((GridCacheDatabaseSharedManager)cctx.database()) + .resolvePageSizeFromPartitionFile(parts.get(0).toPath()); if (pageSize != cctx.database().pageSize()) { throw new IgniteCheckedException("Incompatible memory page size " + From db7b5bb199c2fc573bc7acd8db2109b1d18386dc Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 4 Mar 2021 16:27:36 +0300 Subject: [PATCH 33/98] IGNITE-13805 Code cleanup (remove resolveCacheDir) --- .../snapshot/IgniteSnapshotManager.java | 45 +++++-------------- .../SnapshotRestoreCacheGroupProcess.java | 15 +++++-- .../SnapshotRestoreVerificatioTask.java | 16 ++++--- 3 files changed, 32 insertions(+), 44 deletions(-) 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 cd04ac6a109e72..ec2df251f3e002 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 @@ -162,9 +162,6 @@ import static org.apache.ignite.internal.pagemem.PageIdUtils.pageIndex; import static org.apache.ignite.internal.pagemem.PageIdUtils.toDetailString; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.resolveBinaryWorkDir; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.INDEX_FILE_NAME; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_TEMPLATE; @@ -1106,6 +1103,7 @@ public List readSnapshotMetadatas(String snpName) { /** * @param snpName Snapshot name. * @param grpName Cache group name. + * @param snpCacheDir Cache group directory in snapshot. * @param stopChecker Node stop or prcoess interrupt checker. * @param newFiles A list to keep track of the files created, the list updates during the restore process. * @throws IgniteCheckedException If failed. @@ -1113,16 +1111,12 @@ public List readSnapshotMetadatas(String snpName) { protected void restoreCacheGroupFiles( String snpName, String grpName, + File snpCacheDir, BooleanSupplier stopChecker, List newFiles ) throws IgniteCheckedException { - File snapshotCacheDir = resolveSnapshotCacheDir(snpName, grpName); - - if (!snapshotCacheDir.exists()) - return; - - File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), DFLT_STORE_DIR + - File.separator + pdsSettings.folderName() + File.separator + snapshotCacheDir.getName(), false); + File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), + Paths.get(databaseRelativePath(pdsSettings.folderName()), snpCacheDir.getName()).toString(), false); if (!cacheDir.exists()) { newFiles.add(cacheDir); @@ -1136,20 +1130,21 @@ protected void restoreCacheGroupFiles( } try { - if (log.isInfoEnabled()) { - log.info("Copying partition files of the cache group " + - "[from=" + snapshotCacheDir + ", to=" + cacheDir + ']'); - } + if (log.isInfoEnabled()) + log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); - for (File snpFile : snapshotCacheDir.listFiles()) { + for (File snpFile : snpCacheDir.listFiles()) { if (stopChecker.getAsBoolean()) return; File target = new File(cacheDir, snpFile.getName()); if (log.isDebugEnabled()) { - log.debug("Restoring partition file from the snapshot [snapshot=" + snpName + - ", grp=" + grpName + ", src=" + snpFile + ", target=" + target + "]"); + log.debug("Copying file from the snapshot " + + "[snapshot=" + snpName + + ", grp=" + grpName + + ", src=" + snpFile + + ", target=" + target + "]"); } newFiles.add(target); @@ -1185,22 +1180,6 @@ protected void rollbackRestoreOperation(Collection files) { } } - /** - * @param snpName Snapshot name. - * @param cacheName Cache (group) name. - * @return Local path to the cache directory. - */ - public File resolveSnapshotCacheDir(String snpName, String cacheName) { - File dbDir = new File(snapshotLocalDir(snpName), databaseRelativePath(pdsSettings.folderName())); - - File cacheDir = new File(dbDir, CACHE_DIR_PREFIX + cacheName); - - if (cacheDir.exists()) - return cacheDir; - - return new File(dbDir, CACHE_GRP_DIR_PREFIX + cacheName); - } - /** {@inheritDoc} */ @Override public void onReadyForReadWrite(ReadWriteMetastorage metaStorage) throws IgniteCheckedException { synchronized (snpOpMux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java index e621dea2f54fa0..1a00fc9c794e49 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.cluster.ClusterGroupAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -603,10 +604,10 @@ protected void restore(boolean updateMetadata) throws IgniteCheckedException { return; IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); if (updateMetadata) { - File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), - ctx.pdsFolderResolver().resolveFolders().folderName()); + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), folderName); if (!binDir.exists()) { throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + @@ -616,8 +617,14 @@ protected void restore(boolean updateMetadata) throws IgniteCheckedException { ctx.cacheObjects().updateMetadata(binDir, this::interrupted); } - for (String grpName : groups()) - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, this::interrupted, grps.get(grpName)); + for (File grpDir : snapshotMgr.snapshotCacheDirectories(snpName, folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(grpDir); + + if (!groups().contains(grpName)) + continue; + + snapshotMgr.restoreCacheGroupFiles(snpName, grpName, grpDir, this::interrupted, grps.get(grpName)); + } } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java index 8390e5396e2a53..f9a49a8991729b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java @@ -140,15 +140,16 @@ public SnapshotRestoreVerificationJob(SnapshotRestoreVerificationArg arg) { * @throws IOException In case of I/O errors while reading the memory page size */ private SnapshotRestoreVerificationResult resolveRestoredConfigs() throws IgniteCheckedException, IOException { - GridCacheSharedContext cctx = ignite.context().cache().context(); Map cacheCfgs = new HashMap<>(); + GridCacheSharedContext cctx = ignite.context().cache().context(); + String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); // Collect cache configuration(s) and verify cache groups page size. - for (String grpName : arg.groups()) { - File cacheDir = cctx.snapshotMgr().resolveSnapshotCacheDir(arg.snapshotName(), grpName); + for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(arg.snapshotName(), folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(cacheDir); - if (!cacheDir.exists()) - return null; + if (!arg.groups().contains(grpName)) + continue; ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cacheCfgs); @@ -172,8 +173,9 @@ private SnapshotRestoreVerificationResult resolveRestoredConfigs() throws Ignite if (cacheCfgs.isEmpty()) return null; - File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(arg.snapshotName()).getAbsolutePath(), - ignite.context().pdsFolderResolver().resolveFolders().folderName()); + File binDir = binaryWorkDir( + cctx.snapshotMgr().snapshotLocalDir(arg.snapshotName()).getAbsolutePath(), + folderName); ignite.context().cacheObjects().checkMetadata(binDir); From e8bafdd7a4cae782f45b34e3045ddc4930128855 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 5 Mar 2021 12:36:48 +0300 Subject: [PATCH 34/98] IGNITE-13805 Review notes. --- .../org/apache/ignite/IgniteSnapshot.java | 4 +- .../snapshot/IgniteSnapshotManager.java | 31 +-- .../SnapshotRestoreCacheStartRequest.java | 52 ----- .../SnapshotRestoreEmptyResponse.java | 28 --- ...ocess.java => SnapshotRestoreProcess.java} | 94 +++++--- .../SnapshotRestoreVerificatioTask.java | 185 ---------------- .../SnapshotRestoreVerificationArg.java | 60 ------ .../SnapshotRestoreVerificationTask.java | 200 ++++++++++++++++++ .../IgniteClusterSnapshotRestoreSelfTest.java | 55 ++--- 9 files changed, 297 insertions(+), 412 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestoreCacheGroupProcess.java => SnapshotRestoreProcess.java} (88%) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java create mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index 3023c9ff0d10ac..4d0f5ce3bf2526 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -53,9 +53,9 @@ public interface IgniteSnapshot { /** * Restore cache group(s) from the snapshot. * - * @param snapshotName Snapshot name. + * @param name Snapshot name. * @param cacheGroupNames Cache groups to be restored. * @return Future which will be completed when restore operation finished. */ - public IgniteFuture restoreCacheGroups(String snapshotName, Collection cacheGroupNames); + public IgniteFuture restoreSnapshot(String name, Collection cacheGroupNames); } 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 ec2df251f3e002..551a6292874a82 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 @@ -261,7 +261,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter private final Marshaller marsh; /** Distributed process to restore cache group from the snapshot. */ - private final SnapshotRestoreCacheGroupProcess restoreCacheGrpProc; + private final SnapshotRestoreProcess restoreCacheGrpProc; /** Resolved persistent data storage settings. */ private volatile PdsFolderSettings pdsSettings; @@ -321,7 +321,7 @@ public IgniteSnapshotManager(GridKernalContext ctx) { marsh = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName()); - restoreCacheGrpProc = new SnapshotRestoreCacheGroupProcess(ctx); + restoreCacheGrpProc = new SnapshotRestoreProcess(ctx); } /** @@ -1096,8 +1096,8 @@ public List readSnapshotMetadatas(String snpName) { } /** {@inheritDoc} */ - @Override public IgniteFuture restoreCacheGroups(String snpName, Collection grpNames) { - return restoreCacheGrpProc.start(snpName, grpNames); + @Override public IgniteFuture restoreSnapshot(String name, Collection grpNames) { + return restoreCacheGrpProc.start(name, grpNames); } /** @@ -1157,29 +1157,6 @@ protected void restoreCacheGroupFiles( } } - /** - * @param files Collection of files to delete. - */ - protected void rollbackRestoreOperation(Collection files) { - List dirs = new ArrayList<>(); - - for (File file : files) { - if (!file.exists()) - continue; - - if (file.isDirectory()) - dirs.add(file); - - if (!file.delete()) - log.warning("Unable to delete a file created during a cache restore operation [file=" + file + ']'); - } - - for (File dir : dirs) { - if (!dir.delete()) - log.warning("Unable to delete a folder created during a cache restore operation [file=" + dir + ']'); - } - } - /** {@inheritDoc} */ @Override public void onReadyForReadWrite(ReadWriteMetastorage metaStorage) throws IgniteCheckedException { synchronized (snpOpMux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java deleted file mode 100644 index 8e1ed2d806e505..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheStartRequest.java +++ /dev/null @@ -1,52 +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.Serializable; -import java.util.UUID; -import org.apache.ignite.internal.util.typedef.internal.S; - -/** - * Request to start restored cache group. - */ -public class SnapshotRestoreCacheStartRequest implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Request ID. */ - private final UUID reqId; - - /** - * @param reqId Request ID. - */ - public SnapshotRestoreCacheStartRequest(UUID reqId) { - this.reqId = reqId; - } - - /** - * @return Request ID. - */ - public UUID requestId() { - return reqId; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotRestoreCacheStartRequest.class, this); - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java deleted file mode 100644 index 0cd4bb431739b6..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreEmptyResponse.java +++ /dev/null @@ -1,28 +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.Serializable; - -/** - * Snapshot restore operation single node response. - */ -public class SnapshotRestoreEmptyResponse implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java similarity index 88% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java index 1a00fc9c794e49..1a247b8ec824f0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreCacheGroupProcess.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java @@ -63,7 +63,7 @@ /** * Distributed process to restore cache group from the snapshot. */ -public class SnapshotRestoreCacheGroupProcess { +public class SnapshotRestoreProcess { /** Reject operation message. */ private static final String OP_REJECT_MSG = "Cache group restore operation was rejected. "; @@ -71,10 +71,10 @@ public class SnapshotRestoreCacheGroupProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess prepareRestoreProc; + private final DistributedProcess prepareRestoreProc; /** Cache group restore cache start phase. */ - private final DistributedProcess cacheStartProc; + private final DistributedProcess cacheStartProc; /** Cache group restore rollback phase. */ private final DistributedProcess rollbackRestoreProc; @@ -91,7 +91,7 @@ public class SnapshotRestoreCacheGroupProcess { /** * @param ctx Kernal context. */ - public SnapshotRestoreCacheGroupProcess(GridKernalContext ctx) { + public SnapshotRestoreProcess(GridKernalContext ctx) { this.ctx = ctx; log = ctx.log(getClass()); @@ -121,11 +121,15 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "perform this operation.")); } - IgniteInternalFuture fut0 = fut; + synchronized (this) { + IgniteInternalFuture fut0 = fut; - if (!fut0.isDone()) { - return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + - "The previous snapshot restore operation was not completed.")); + if (!fut0.isDone()) { + return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + + "The previous snapshot restore operation was not completed.")); + } + + fut = new GridFutureAdapter<>(); } DiscoveryDataClusterState clusterState = ctx.state().clusterState(); @@ -151,15 +155,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames Set bltNodeIds = new HashSet<>(F.viewReadOnly(bltNodes, F.node2id())); - fut = new GridFutureAdapter<>(); - ((ClusterGroupAdapter)ctx.cluster().get().forNodeIds(bltNodeIds)).compute().executeAsync( - new SnapshotRestoreVerificatioTask(), new SnapshotRestoreVerificationArg(snpName, cacheGrpNames)).listen( + new SnapshotRestoreVerificationTask(snpName, cacheGrpNames), null).listen( f -> { try { - SnapshotRestoreVerificationResult res = f.get(); + Map.Entry> firstNodeRes = F.first(f.get().entrySet()); - Set foundGrps = res == null ? Collections.emptySet() : res.configs().stream() + Set foundGrps = firstNodeRes == null ? Collections.emptySet() : firstNodeRes.getValue().stream() .map(v -> v.config().getGroupName() != null ? v.config().getGroupName() : v.config().getName()) .collect(Collectors.toSet()); @@ -174,8 +176,12 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest( - UUID.randomUUID(), snpName, bltNodeIds, res.configs(), res.localNodeId()); + HashSet reqNodes = new HashSet<>(f.get().keySet()); + + reqNodes.add(ctx.localNodeId()); + + SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, + reqNodes, firstNodeRes.getValue(), firstNodeRes.getKey()); prepareRestoreProc.start(req.requestId(), req); } catch (Throwable t) { @@ -237,9 +243,8 @@ public void stop(Exception reason) { * Ensures that a cache with the specified name does not exist locally. * * @param name Cache name. - * @throws IllegalStateException If cache with the specified name already exists. */ - private void ensureCacheAbsent(String name) throws IllegalStateException { + private void ensureCacheAbsent(String name) { int id = CU.cacheId(name); if (ctx.cache().cacheGroupDescriptors().containsKey(id) || ctx.cache().cacheDescriptor(id) != null) { @@ -252,7 +257,7 @@ private void ensureCacheAbsent(String name) throws IllegalStateException { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { + private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { if (!req.nodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); @@ -279,7 +284,7 @@ private IgniteInternalFuture prepare(SnapshotResto SnapshotRestoreContext opCtx0 = opCtx; - GridFutureAdapter retFut = new GridFutureAdapter<>(); + GridFutureAdapter retFut = new GridFutureAdapter<>(); try { for (String grpName : opCtx0.groups()) @@ -300,7 +305,7 @@ private IgniteInternalFuture prepare(SnapshotResto opCtx0.restore(updateMeta); if (!opCtx0.interrupted()) { - retFut.onDone(); + retFut.onDone(true); return; } @@ -329,14 +334,23 @@ private IgniteInternalFuture prepare(SnapshotResto * @param res Results. * @param errs Errors. */ - private void finishPrepare(UUID reqId, Map res, Map errs) { + private void finishPrepare(UUID reqId, Map res, Map errs) { GridFutureAdapter fut0 = fut; + SnapshotRestoreContext opCtx0 = opCtx; - if (fut0.isDone() || !reqId.equals(opCtx.requestId())) + if (staleProcess(fut0, opCtx)) return; Exception failure = F.first(errs.values()); + if (failure == null && !res.keySet().containsAll(opCtx0.nodes())) { + Set leftNodes = new HashSet<>(opCtx0.nodes()); + + leftNodes.removeAll(res.keySet()); + + failure = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); + } + if (failure != null) { opCtx.rollback(); @@ -346,20 +360,20 @@ private void finishPrepare(UUID reqId, Map r } if (U.isLocalNodeCoordinator(ctx.discovery())) - cacheStartProc.start(reqId, new SnapshotRestoreCacheStartRequest(reqId)); + cacheStartProc.start(reqId, reqId); } /** - * @param req Request to start restored cache groups. + * @param reqId Request ID. * @return Result future. */ - private IgniteInternalFuture cacheStart(SnapshotRestoreCacheStartRequest req) { + private IgniteInternalFuture cacheStart(UUID reqId) { SnapshotRestoreContext opCtx0 = opCtx; if (staleProcess(fut, opCtx0)) return new GridFinishedFuture<>(); - if (!req.requestId().equals(opCtx0.requestId())) + if (!reqId.equals(opCtx0.requestId())) return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); if (!U.isLocalNodeCoordinator(ctx.discovery())) @@ -376,7 +390,7 @@ private IgniteInternalFuture cacheStart(SnapshotRe if (!allNodesInBaselineAndAlive(opCtx0.nodes())) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); - GridFutureAdapter retFut = new GridFutureAdapter<>(); + GridFutureAdapter retFut = new GridFutureAdapter<>(); if (log.isInfoEnabled()) { log.info("Starting restored caches " + @@ -392,7 +406,7 @@ private IgniteInternalFuture cacheStart(SnapshotRe retFut.onDone(f.error()); } else - retFut.onDone(); + retFut.onDone(true); } ); @@ -404,7 +418,7 @@ private IgniteInternalFuture cacheStart(SnapshotRe * @param res Results. * @param errs Errors. */ - private void finishCacheStart(UUID reqId, Map res, Map errs) { + private void finishCacheStart(UUID reqId, Map res, Map errs) { GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; @@ -639,8 +653,26 @@ protected void rollback() { for (String grpName : grpNames) { List files = grps.remove(grpName); - if (files != null) - ctx.cache().context().snapshotMgr().rollbackRestoreOperation(files); + if (files == null) + continue; + + List dirs = new ArrayList<>(); + + for (File file : files) { + if (!file.exists()) + continue; + + if (file.isDirectory()) + dirs.add(file); + + if (!file.delete()) + log.warning("Unable to delete a file created during a cache restore operation [file=" + file + ']'); + } + + for (File dir : dirs) { + if (!dir.delete()) + log.warning("Unable to delete a folder created during a cache restore operation [file=" + dir + ']'); + } } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java deleted file mode 100644 index f9a49a8991729b..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificatioTask.java +++ /dev/null @@ -1,185 +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.io.IOException; -import java.util.ArrayList; -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.binary.BinaryObjectException; -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.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager; -import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.jetbrains.annotations.NotNull; - -import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; - -/** - * Verification task for restoring a cache group from a snapshot. - */ -public class SnapshotRestoreVerificatioTask extends - ComputeTaskAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** {@inheritDoc} */ - @Override public @NotNull Map map(List subgrid, - SnapshotRestoreVerificationArg arg) throws IgniteException { - Map jobs = new HashMap<>(); - - for (ClusterNode node : subgrid) - jobs.put(new SnapshotRestoreVerificationJob(arg), node); - - return jobs; - } - - /** {@inheritDoc} */ - @Override public SnapshotRestoreVerificationResult reduce(List results) throws IgniteException { - SnapshotRestoreVerificationResult firstRes = null; - - for (ComputeJobResult jobRes : results) { - SnapshotRestoreVerificationResult res = jobRes.getData(); - - if (res == null) - continue; - - if (firstRes == null) { - firstRes = res; - - continue; - } - - if (firstRes.configs().size() != res.configs().size()) { - throw new IgniteException("Count of cache configs mismatch [" + - "node1=" + firstRes.localNodeId() + ", cnt1=" + firstRes.configs().size() + - ", node2=" + res.localNodeId() + ", cnt2=" + res.configs().size() + ']'); - } - } - - return firstRes; - } - - /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) { - IgniteException e = res.getException(); - - // Don't failover this job, if topology changed - user should restart operation. - if (e != null) - throw e; - - return super.result(res, rcvd); - } - - /** */ - private static class SnapshotRestoreVerificationJob extends ComputeJobAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Auto-injected grid instance. */ - @IgniteInstanceResource - private transient IgniteEx ignite; - - /** Job argument. */ - private final SnapshotRestoreVerificationArg arg; - - /** - * @param arg Job argument. - */ - public SnapshotRestoreVerificationJob(SnapshotRestoreVerificationArg arg) { - this.arg = arg; - } - - /** {@inheritDoc} */ - @Override public Object execute() throws IgniteException { - assert !ignite.context().clientNode(); - - try { - return resolveRestoredConfigs(); - } - catch (BinaryObjectException e) { - throw new IgniteException("Incompatible binary types found: " + e.getMessage()); - } catch (IOException | IgniteCheckedException e) { - throw F.wrap(e); - } - } - - /** - * Collect cache configurations and verify binary compatibility of specified cache groups. - * - * @return List of stored cache configurations with local node ID. - * @throws IgniteCheckedException If the snapshot is incompatible. - * @throws IOException In case of I/O errors while reading the memory page size - */ - private SnapshotRestoreVerificationResult resolveRestoredConfigs() throws IgniteCheckedException, IOException { - Map cacheCfgs = new HashMap<>(); - GridCacheSharedContext cctx = ignite.context().cache().context(); - String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); - - // Collect cache configuration(s) and verify cache groups page size. - for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(arg.snapshotName(), folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(cacheDir); - - if (!arg.groups().contains(grpName)) - continue; - - ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cacheCfgs); - - List parts = FilePageStoreManager.cachePartitionFiles(cacheDir); - - if (F.isEmpty(parts)) - continue; - - int pageSize = ((GridCacheDatabaseSharedManager)cctx.database()) - .resolvePageSizeFromPartitionFile(parts.get(0).toPath()); - - if (pageSize != cctx.database().pageSize()) { - throw new IgniteCheckedException("Incompatible memory page size " + - "[snapshotPageSize=" + pageSize + - ", nodePageSize=" + cctx.database().pageSize() + - ", group=" + grpName + - ", snapshot=" + arg.snapshotName() + ']'); - } - } - - if (cacheCfgs.isEmpty()) - return null; - - File binDir = binaryWorkDir( - cctx.snapshotMgr().snapshotLocalDir(arg.snapshotName()).getAbsolutePath(), - folderName); - - ignite.context().cacheObjects().checkMetadata(binDir); - - return new SnapshotRestoreVerificationResult(new ArrayList<>(cacheCfgs.values()), ignite.localNode().id()); - } - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java deleted file mode 100644 index e7c2f01ee29856..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationArg.java +++ /dev/null @@ -1,60 +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.Serializable; -import java.util.Collection; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; - -/** - * Verification task argument. - */ -public class SnapshotRestoreVerificationArg implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Snapshot name. */ - private final String snpName; - - /** List of cache group names to restore from the snapshot. */ - @GridToStringInclude - private final Collection grps; - - /** - * @param snpName Snapshot name. - * @param grps List of cache group names to restore from the snapshot. - */ - public SnapshotRestoreVerificationArg(String snpName, Collection grps) { - this.snpName = snpName; - this.grps = grps; - } - - /** - * @return List of cache group names to restore from the snapshot. - */ - public Collection groups() { - return grps; - } - - /** - * @return Snapshot name. - */ - public String snapshotName() { - return snpName; - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java new file mode 100644 index 00000000000000..a8914fe3c56da2 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java @@ -0,0 +1,200 @@ +/* + * 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.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.ignite.IgniteCheckedException; +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.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.StoredCacheData; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.jetbrains.annotations.NotNull; + +import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; + +/** + * Verification task for restoring a cache group from a snapshot. + */ +public class SnapshotRestoreVerificationTask extends ComputeTaskAdapter>> { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Snapshot name. */ + private final String snpName; + + /** List of cache group names to restore from the snapshot. */ + @GridToStringInclude + private final Collection grps; + + /** + * @param snpName Snapshot name. + * @param grps List of cache group names to restore from the snapshot. + */ + public SnapshotRestoreVerificationTask(String snpName, Collection grps) { + this.snpName = snpName; + this.grps = grps; + } + + /** {@inheritDoc} */ + @Override public @NotNull Map map(List subgrid, + Void arg) throws IgniteException { + Map jobs = new HashMap<>(); + + for (ClusterNode node : subgrid) + jobs.put(new SnapshotRestoreVerificationJob(snpName, grps), node); + + return jobs; + } + + /** {@inheritDoc} */ + @Override public Map> reduce(List results) throws IgniteException { + Map> resMap = new HashMap<>(); + ComputeJobResult firstNodeRes = null; + + for (ComputeJobResult jobRes : results) { + List res = jobRes.getData(); + + if (res == null) + continue; + + resMap.put(jobRes.getNode().id(), res); + + if (firstNodeRes == null) { + firstNodeRes = jobRes; + + continue; + } + + int expCfgCnt = ((Collection)firstNodeRes.getData()).size(); + + if (expCfgCnt != res.size()) { + throw new IgniteException("Count of cache configs mismatch [" + + "node1=" + firstNodeRes.getNode().id() + ", cnt1=" + expCfgCnt + + ", node2=" + jobRes.getNode().id() + ", cnt2=" + res.size() + ']'); + } + } + + return resMap; + } + + /** {@inheritDoc} */ + @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) { + IgniteException e = res.getException(); + + // Don't failover this job, if topology changed - user should restart operation. + if (e != null) + throw e; + + return super.result(res, rcvd); + } + + /** */ + private static class SnapshotRestoreVerificationJob extends ComputeJobAdapter { + /** Serial version uid. */ + private static final long serialVersionUID = 0L; + + /** Auto-injected grid instance. */ + @IgniteInstanceResource + private transient IgniteEx ignite; + + /** Snapshot name. */ + private final String snpName; + + /** List of cache group names to restore from the snapshot. */ + @GridToStringInclude + private final Collection grps; + + /** + * @param snpName Snapshot name. + * @param grps List of cache group names to restore from the snapshot. + */ + public SnapshotRestoreVerificationJob(String snpName, Collection grps) { + this.snpName = snpName; + this.grps = grps; + } + + /** {@inheritDoc} */ + @Override public Object execute() throws IgniteException { + assert !ignite.context().clientNode(); + + try { + Map ccfgs = new HashMap<>(); + GridCacheSharedContext cctx = ignite.context().cache().context(); + String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); + + List metas = cctx.snapshotMgr().readSnapshotMetadatas(snpName); + + if (F.isEmpty(metas)) + return null; + + SnapshotMetadata meta = metas.get(0); + + if (!meta.consistentId().equals(cctx.localNode().consistentId().toString())) + return null; + + if (meta.pageSize() != cctx.database().pageSize()) { + throw new IgniteCheckedException("Incompatible memory page size " + + "[snapshotPageSize=" + meta.pageSize() + + ", local=" + cctx.database().pageSize() + + ", snapshot=" + snpName + + ", nodeId=" + cctx.localNodeId() + ']'); + } + + // Collect cache configuration(s) and verify cache groups page size. + for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(snpName, folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(cacheDir); + + if (!grps.contains(grpName)) + continue; + + ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, ccfgs); + } + + if (ccfgs.isEmpty()) + return null; + + File binDir = binaryWorkDir( + cctx.snapshotMgr().snapshotLocalDir(snpName).getAbsolutePath(), + folderName); + + ignite.context().cacheObjects().checkMetadata(binDir); + + return new ArrayList<>(ccfgs.values()); + } + catch (IgniteCheckedException e) { + throw new IgniteException(e); + } + } + } +} 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 7755516a786e64..e4fda45ace8138 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 @@ -31,6 +31,7 @@ import org.apache.ignite.IgniteSnapshot; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.CacheMode; @@ -122,7 +123,7 @@ public void testCacheStartFailOnNodeLeft() throws Exception { discoSpi.block((msg) -> msg instanceof DynamicCacheChangeBatch); IgniteFuture fut = - grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); discoSpi.waitBlocked(TIMEOUT); @@ -142,7 +143,7 @@ public void testBasicClusterSnapshotRestore() throws Exception { IgniteEx ignite = startGridsWithSnapshot(2, keysCnt, true); - grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); @@ -167,7 +168,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { forceCheckpoint(); - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); @@ -186,7 +187,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception ignite.cluster().state(ClusterState.INACTIVE); IgniteFuture fut = - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); @@ -201,12 +202,13 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); - IgniteEx ignite = startGridsWithCache(nodesCnt - 2, keysCnt, valBuilder, dfltCacheCfg); + startGridsWithCache(nodesCnt - 2, keysCnt, valBuilder, dfltCacheCfg); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); + grid(0).snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); startGrid(nodesCnt - 2); - startGrid(nodesCnt - 1); + + IgniteEx ignite = startGrid(nodesCnt - 1); resetBaselineTopology(); @@ -217,16 +219,17 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { awaitPartitionMapExchange(); // remove metadata - int typeId = grid(nodesCnt - 1).context().cacheObjects().typeId(BIN_TYPE_NAME); + int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); - grid(nodesCnt - 1).context().cacheObjects().removeType(typeId); + ignite.context().cacheObjects().removeType(typeId); forceCheckpoint(); - ignite.snapshot().restoreCacheGroups( + // Restore from an empty node. + ignite.snapshot().restoreSnapshot( SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - IgniteCache cache = grid(nodesCnt - 1).cache(dfltCacheCfg.getName()).withKeepBinary(); + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); assertTrue(cache.indexReadyFuture().isDone()); @@ -267,7 +270,7 @@ public void testRestoreSharedCacheGroup() throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> snp.restoreCacheGroups(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(TIMEOUT), + () -> snp.restoreSnapshot(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(TIMEOUT), IllegalArgumentException.class, "Cache group(s) was not found in the snapshot" ); @@ -276,7 +279,7 @@ public void testRestoreSharedCacheGroup() throws Exception { awaitPartitionMapExchange(); - snp.restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)).get(TIMEOUT); + snp.restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(grpName)).get(TIMEOUT); checkCacheKeys(ignite.cache(cacheName1), CACHE_KEYS_RANGE); checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); @@ -288,8 +291,11 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { String cacheName1 = "cache1"; String cacheName2 = "cache2"; - CacheConfiguration cacheCfg1 = txCacheConfig(new CacheConfiguration(cacheName1)).setCacheMode(CacheMode.REPLICATED); - CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration(cacheName2)).setCacheMode(CacheMode.REPLICATED); + CacheConfiguration cacheCfg1 = + txCacheConfig(new CacheConfiguration(cacheName1)).setCacheMode(CacheMode.REPLICATED); + + CacheConfiguration cacheCfg2 = + txCacheConfig(new CacheConfiguration(cacheName2)).setCacheMode(CacheMode.REPLICATED); IgniteEx ignite0 = startGrid(0); IgniteEx ignite1 = startGrid(1); @@ -328,10 +334,10 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { U.delete(resolveCacheDir(grid(nodeIdx), cacheName)); } - ignite0.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); + ignite0.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); awaitPartitionMapExchange(); - ignite1.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); + ignite1.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); awaitPartitionMapExchange(); checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); @@ -362,7 +368,7 @@ public void testIncompatibleMetasUpdate() throws Exception { }); IgniteFuture fut = - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); fut.get(TIMEOUT); @@ -392,14 +398,9 @@ public void testIncompatibleMetasUpdate() throws Exception { }); IgniteFuture fut0 = - ignite.snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); - GridTestUtils.assertThrowsAnyCause( - log, - () -> fut0.get(TIMEOUT), - IgniteException.class, - "Incompatible binary types found" - ); + GridTestUtils.assertThrowsAnyCause(log, () -> fut0.get(TIMEOUT), BinaryObjectException.class, null); ensureCacheDirEmpty(2, dfltCacheCfg.getName()); @@ -651,7 +652,7 @@ private void checkClusterStateChange( resetBaselineTopology(); } - grid(nodesCnt - 1).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); + grid(nodesCnt - 1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); checkCacheKeys(ignite.cache(cacheName), CACHE_KEYS_RANGE); } @@ -749,7 +750,7 @@ private IgniteFuture waitForBlockOnRestore( msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == restorePhase.ordinal()); IgniteFuture fut = - grid(0).snapshot().restoreCacheGroups(SNAPSHOT_NAME, Collections.singleton(grpName)); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(grpName)); spi.waitForBlocked(); From 9c5f55d1f58ff990f3540feeb70f9f47a1b02a72 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 10 Mar 2021 17:56:41 +0300 Subject: [PATCH 35/98] IGNITE-13805 Remove verification task, --- .../snapshot/IgniteSnapshotManager.java | 121 ++-- .../SnapshotRestorePrepareRequest.java | 25 +- .../snapshot/SnapshotRestoreProcess.java | 530 +++++++++++------- .../SnapshotRestoreVerificationResult.java | 60 -- .../SnapshotRestoreVerificationTask.java | 200 ------- .../IgniteClusterSnapshotRestoreSelfTest.java | 24 +- 6 files changed, 410 insertions(+), 550 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.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 551a6292874a82..3b60d375e818ed 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 @@ -55,7 +55,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; -import java.util.function.BooleanSupplier; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; @@ -858,34 +857,20 @@ public IgniteInternalFuture checkSnapshot(String name) { 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_"); - GridKernalContext kctx0 = cctx.kernalContext(); GridFutureAdapter res = new GridFutureAdapter<>(); - kctx0.security().authorize(ADMIN_SNAPSHOT); - - Collection bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - (node) -> CU.baselineNode(node, kctx0.state().clusterState())); - - kctx0.task().setThreadContext(TC_SKIP_AUTH, true); - kctx0.task().setThreadContext(TC_SUBGRID, bltNodes); - - kctx0.task().execute(SnapshotMetadataCollectorTask.class, name) - .listen(f0 -> { + collectSnapshotMetadata(name).listen(f0 -> { if (f0.error() == null) { Map> metas = f0.result(); - kctx0.task().setThreadContext(TC_SKIP_AUTH, true); - kctx0.task().setThreadContext(TC_SUBGRID, new ArrayList<>(metas.keySet())); - - kctx0.task().execute(SnapshotPartitionsVerifyTask.class, metas) - .listen(f1 -> { - if (f1.error() == null) - res.onDone(f1.result()); - else if (f1.error() instanceof IgniteSnapshotVerifyException) - res.onDone(new IdleVerifyResultV2(((IgniteSnapshotVerifyException)f1.error()).exceptions())); - else - res.onDone(f1.error()); - }); + runSnapshotVerfification(metas).listen(f1 -> { + if (f1.error() == null) + res.onDone(f1.result()); + else if (f1.error() instanceof IgniteSnapshotVerifyException) + res.onDone(new IdleVerifyResultV2(((IgniteSnapshotVerifyException)f1.error()).exceptions())); + else + res.onDone(f1.error()); + }); } else { if (f0.error() instanceof IgniteSnapshotVerifyException) @@ -898,6 +883,37 @@ else if (f1.error() instanceof IgniteSnapshotVerifyException) return res; } + /** + * @param name Snapshot name. + * @return Future with snapshot metadata obtained from nodes. + */ + IgniteInternalFuture>> collectSnapshotMetadata(String name) { + GridKernalContext kctx0 = cctx.kernalContext(); + + kctx0.security().authorize(ADMIN_SNAPSHOT); + + Collection bltNodes = F.view(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + (node) -> CU.baselineNode(node, kctx0.state().clusterState())); + + kctx0.task().setThreadContext(TC_SKIP_AUTH, true); + kctx0.task().setThreadContext(TC_SUBGRID, bltNodes); + + return kctx0.task().execute(SnapshotMetadataCollectorTask.class, name); + } + + /** + * @param metas Nodes snapshot metadata. + * @return Future with the verification results. + */ + IgniteInternalFuture runSnapshotVerfification(Map> metas) { + GridKernalContext kctx0 = cctx.kernalContext(); + + kctx0.task().setThreadContext(TC_SKIP_AUTH, true); + kctx0.task().setThreadContext(TC_SUBGRID, new ArrayList<>(metas.keySet())); + + return kctx0.task().execute(SnapshotPartitionsVerifyTask.class, metas); + } + /** * @param snpName Snapshot name. * @param folderName Directory name for cache group. @@ -1100,63 +1116,6 @@ public List readSnapshotMetadatas(String snpName) { return restoreCacheGrpProc.start(name, grpNames); } - /** - * @param snpName Snapshot name. - * @param grpName Cache group name. - * @param snpCacheDir Cache group directory in snapshot. - * @param stopChecker Node stop or prcoess interrupt checker. - * @param newFiles A list to keep track of the files created, the list updates during the restore process. - * @throws IgniteCheckedException If failed. - */ - protected void restoreCacheGroupFiles( - String snpName, - String grpName, - File snpCacheDir, - BooleanSupplier stopChecker, - List newFiles - ) throws IgniteCheckedException { - File cacheDir = U.resolveWorkDirectory(cctx.kernalContext().config().getWorkDirectory(), - Paths.get(databaseRelativePath(pdsSettings.folderName()), snpCacheDir.getName()).toString(), false); - - if (!cacheDir.exists()) { - newFiles.add(cacheDir); - - cacheDir.mkdir(); - } - else - if (cacheDir.list().length > 0) { - throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + - "[group=" + grpName + ", dir=" + cacheDir + ']'); - } - - try { - if (log.isInfoEnabled()) - log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); - - for (File snpFile : snpCacheDir.listFiles()) { - if (stopChecker.getAsBoolean()) - return; - - File target = new File(cacheDir, snpFile.getName()); - - if (log.isDebugEnabled()) { - log.debug("Copying file from the snapshot " + - "[snapshot=" + snpName + - ", grp=" + grpName + - ", src=" + snpFile + - ", target=" + target + "]"); - } - - newFiles.add(target); - - Files.copy(snpFile.toPath(), target.toPath()); - } - } - catch (IOException e) { - throw new IgniteCheckedException("Unable to copy file [snapshot=" + snpName + ", grp=" + grpName + ']', e); - } - } - /** {@inheritDoc} */ @Override public void onReadyForReadWrite(ReadWriteMetastorage metaStorage) throws IgniteCheckedException { synchronized (snpOpMux) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index 4ffd104f5efae6..cd7df1dbd6137b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -18,10 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import java.io.Serializable; -import java.util.List; +import java.util.Collection; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; @@ -43,30 +42,30 @@ public class SnapshotRestorePrepareRequest implements Serializable { /** Stored cache configurations. */ @GridToStringExclude - private final List ccfgs; + private final Collection grps; /** Node ID from which to update the binary metadata. */ - private final UUID updateMetaNodeId; + private final UUID performNodeId; /** * @param reqId Request ID. * @param snpName Snapshot name. * @param nodes Baseline node IDs that must be alive to complete the operation. * @param ccfgs Stored cache configurations. - * @param updateMetaNodeId Node ID from which to update the binary metadata. + * @param performNodeId Node ID from which to update the binary metadata. */ public SnapshotRestorePrepareRequest( UUID reqId, String snpName, Set nodes, - List ccfgs, - UUID updateMetaNodeId + Collection grps, + UUID performNodeId ) { this.reqId = reqId; this.snpName = snpName; this.nodes = nodes; - this.ccfgs = ccfgs; - this.updateMetaNodeId = updateMetaNodeId; + this.grps = grps; + this.performNodeId = performNodeId; } /** @@ -86,8 +85,8 @@ public String snapshotName() { /** * @return Stored cache configurations. */ - public List configs() { - return ccfgs; + public Collection groups() { + return grps; } /** @@ -100,8 +99,8 @@ public Set nodes() { /** * @return Node ID from which to update the binary metadata. */ - public UUID updateMetaNodeId() { - return updateMetaNodeId; + public UUID performNodeId() { + return performNodeId; } /** {@inheritDoc} */ 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 1a247b8ec824f0..f52617cc0e8db0 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 @@ -1,34 +1,41 @@ -/* - * 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. - */ +///* +// * 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.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; +import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Queue; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -38,11 +45,12 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.cluster.ClusterGroupAdapter; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; +import org.apache.ignite.internal.util.GridConcurrentHashSet; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -56,6 +64,7 @@ import static org.apache.ignite.internal.IgniteFeatures.SNAPSHOT_RESTORE_CACHE_GROUP; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.databaseRelativePath; 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_ROLLBACK; import static org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType.RESTORE_CACHE_GROUP_SNAPSHOT_START; @@ -71,7 +80,7 @@ public class SnapshotRestoreProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess prepareRestoreProc; + private final DistributedProcess> prepareRestoreProc; /** Cache group restore cache start phase. */ private final DistributedProcess cacheStartProc; @@ -142,7 +151,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames "The baseline topology is not configured for cluster.")); } - if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) { + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + + if (snpMgr.isSnapshotCreating()) { return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); } @@ -150,43 +161,66 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) throw new IgniteException("Not all nodes in the cluster support a snapshot restore operation."); - Collection bltNodes = F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), - node -> node, (node) -> CU.baselineNode(node, ctx.state().clusterState())); - - Set bltNodeIds = new HashSet<>(F.viewReadOnly(bltNodes, F.node2id())); - - ((ClusterGroupAdapter)ctx.cluster().get().forNodeIds(bltNodeIds)).compute().executeAsync( - new SnapshotRestoreVerificationTask(snpName, cacheGrpNames), null).listen( + snpMgr.collectSnapshotMetadata(snpName).listen( f -> { - try { - Map.Entry> firstNodeRes = F.first(f.get().entrySet()); + if (f.error() != null) { + fut.onDone(f.error()); - Set foundGrps = firstNodeRes == null ? Collections.emptySet() : firstNodeRes.getValue().stream() - .map(v -> v.config().getGroupName() != null ? v.config().getGroupName() : v.config().getName()) - .collect(Collectors.toSet()); + return; + } - if (!foundGrps.containsAll(cacheGrpNames)) { - Set missedGroups = new HashSet<>(cacheGrpNames); + Set dataNodes = new LinkedHashSet<>(); + Map> metas = f.result(); + Map reqGrpIds = cacheGrpNames.stream().collect(Collectors.toMap(CU::cacheId, v -> v)); - missedGroups.removeAll(foundGrps); + for (Map.Entry> entry : metas.entrySet()) { + SnapshotMetadata meta = F.first(entry.getValue()); - fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + - "snapshot [groups=" + missedGroups + ", snapshot=" + snpName + ']')); + assert meta != null : entry.getKey().id(); - return; - } + if (!entry.getKey().consistentId().equals(meta.consistentId())) + continue; - HashSet reqNodes = new HashSet<>(f.get().keySet()); + dataNodes.add(entry.getKey().id()); - reqNodes.add(ctx.localNodeId()); + reqGrpIds.keySet().removeAll(meta.partitions().keySet()); + } - SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), snpName, - reqNodes, firstNodeRes.getValue(), firstNodeRes.getKey()); + if (!reqGrpIds.isEmpty()) { + fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); - prepareRestoreProc.start(req.requestId(), req); - } catch (Throwable t) { - fut.onDone(new IgniteException(OP_REJECT_MSG + t.getMessage(), t)); + return; } + + dataNodes.add(ctx.localNodeId()); + + snpMgr.runSnapshotVerfification(metas).listen( + f0 -> { + if (f0.error() != null) { + fut.onDone(f0.error()); + + return; + } + + IdleVerifyResultV2 res = f0.result(); + + if (!F.isEmpty(res.exceptions()) || res.hasConflicts()) { + StringBuilder sb = new StringBuilder(); + + res.print(sb::append, true); + + fut.onDone(new IgniteException(sb.toString())); + + return; + } + + SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), + snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); + + prepareRestoreProc.start(req.requestId(), req); + } + ); } ); @@ -257,8 +291,8 @@ private void ensureCacheAbsent(String name) { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) { - if (!req.nodes().contains(ctx.localNodeId())) + private IgniteInternalFuture> prepare(SnapshotRestorePrepareRequest req) { + if (ctx.clientNode()) return new GridFinishedFuture<>(); if (inProgress(null)) { @@ -275,37 +309,63 @@ private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) if (fut.isDone()) fut = new GridFutureAdapter<>(); - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), req.configs()); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.groups(), req.nodes()); fut.listen(f -> opCtx = null); - if (!allNodesInBaselineAndAlive(req.nodes())) - return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); + try { + Map cfgMap = new HashMap<>(); + GridCacheSharedContext cctx = ctx.cache().context(); + String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - SnapshotRestoreContext opCtx0 = opCtx; + SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); - GridFutureAdapter retFut = new GridFutureAdapter<>(); + if (meta == null || !meta.consistentId().equals(cctx.localNode().consistentId().toString())) + return new GridFinishedFuture<>(); - try { - for (String grpName : opCtx0.groups()) - ensureCacheAbsent(grpName); + if (meta.pageSize() != cctx.database().pageSize()) { + throw new IgniteCheckedException("Incompatible memory page size " + + "[snapshotPageSize=" + meta.pageSize() + + ", local=" + cctx.database().pageSize() + + ", snapshot=" + req.snapshotName() + + ", nodeId=" + cctx.localNodeId() + ']'); + } - for (StoredCacheData cfg : opCtx0.configs()) { - if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); + SnapshotRestoreContext opCtx0 = opCtx; + + // Collect cache configuration(s) and verify cache groups page size. + for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(cacheDir); + + if (!opCtx0.groups().contains(grpName)) + continue; + + ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cfgMap); } - if (!ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx0.snapshotName()).exists()) + ArrayList ccfgs = new ArrayList<>(cfgMap.values()); + + if (ccfgs.isEmpty()) return new GridFinishedFuture<>(); - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + if (!allNodesInBaselineAndAlive(req.nodes())) + throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + + for (String grpName : opCtx0.groups()) + ensureCacheAbsent(grpName); + + File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(req.snapshotName()).getAbsolutePath(), folderName); + + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - ctx.getSystemExecutorService().submit(() -> { + cctx.snapshotMgr().snapshotExecutorService().execute(() -> { try { - opCtx0.restore(updateMeta); + ctx.cacheObjects().checkMetadata(binDir); + + restore(opCtx0, binDir, ctx.localNodeId().equals(req.performNodeId())); if (!opCtx0.interrupted()) { - retFut.onDone(true); + retFut.onDone(ccfgs); return; } @@ -313,7 +373,7 @@ private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) log.error("Snapshot restore process has been interrupted " + "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', opCtx0.error()); - opCtx0.rollback(); + rollback(opCtx0); retFut.onDone(opCtx0.error()); @@ -325,42 +385,142 @@ private IgniteInternalFuture prepare(SnapshotRestorePrepareRequest req) return retFut; } catch (Exception e) { + log.error("Unable to restore cache group(s) from snapshot " + + "[groups=" + req.groups() + ", snapshot=" + req.snapshotName() + ']', e); + return new GridFinishedFuture<>(e); } } + /** + * Restore specified cache groups from the local snapshot directory. + * + * @param updateMetadata Update binary metadata flag. + * @throws IgniteCheckedException If failed. + */ + protected void restore(SnapshotRestoreContext opCtx, File binDir, boolean updateMetadata) throws IgniteCheckedException { + if (opCtx.interrupted()) + return; + + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); + + if (updateMetadata) { + assert binDir.exists(); + + ctx.cacheObjects().updateMetadata(binDir, opCtx::interrupted); + } + + for (File snpCacheDir : snapshotMgr.snapshotCacheDirectories(opCtx.snapshotName(), folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); + + if (!opCtx.groups().contains(grpName)) + continue; + + File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), + Paths.get(databaseRelativePath(folderName), snpCacheDir.getName()).toString(), false); + + if (!cacheDir.exists()) + cacheDir.mkdir(); + else + if (cacheDir.list().length > 0) { + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } + + opCtx.pushDir(cacheDir); + + restoreCacheGroupFiles(opCtx.snapshotName(), grpName, cacheDir, snpCacheDir, opCtx::interrupted); + } + } + + /** + * @param snpName Snapshot name. + * @param grpName Cache group name. + * @param cacheDir Cache group directory. + * @param snpCacheDir Cache group directory in snapshot. + * @param stopChecker Node stop or prcoess interrupt checker. + * @throws IgniteCheckedException If failed. + */ + protected void restoreCacheGroupFiles(String snpName, String grpName, File cacheDir, File snpCacheDir, + BooleanSupplier stopChecker) throws IgniteCheckedException { + try { + if (log.isInfoEnabled()) + log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); + + for (File snpFile : snpCacheDir.listFiles()) { + if (stopChecker.getAsBoolean()) + return; + + File target = new File(cacheDir, snpFile.getName()); + + if (log.isDebugEnabled()) { + log.debug("Copying file from the snapshot " + + "[snapshot=" + snpName + + ", grp=" + grpName + + ", src=" + snpFile + + ", target=" + target + "]"); + } + + Files.copy(snpFile.toPath(), target.toPath()); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to copy file [snapshot=" + snpName + ", grp=" + grpName + ']', e); + } + } + + /** + * Rollback changes made by process in specified cache group. + */ + protected void rollback(SnapshotRestoreContext opCtx) { + File rmvDir; + + while ((rmvDir = opCtx.popDir()) != null) { + if (!U.delete(rmvDir)) + log.error("Unable to delete restored cache directory [dir=" + rmvDir + ']'); + } + } + /** * @param reqId Request ID. * @param res Results. * @param errs Errors. */ - private void finishPrepare(UUID reqId, Map res, Map errs) { + private void finishPrepare(UUID reqId, Map> res, Map errs) { GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut0, opCtx)) + if (staleProcess(fut0, opCtx0)) return; - Exception failure = F.first(errs.values()); + Exception failure = checkFailure(errs, opCtx0.nodes(), res.keySet()); - if (failure == null && !res.keySet().containsAll(opCtx0.nodes())) { - Set leftNodes = new HashSet<>(opCtx0.nodes()); + if (failure == null) { + Map filteredCfgs = new HashMap<>(); - leftNodes.removeAll(res.keySet()); + for (List storedCfgs : res.values()) { + if (storedCfgs == null) + continue; - failure = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); - } + for (StoredCacheData cacheData : storedCfgs) + filteredCfgs.put(cacheData.config().getName(), cacheData); + } - if (failure != null) { - opCtx.rollback(); + opCtx.configs(filteredCfgs.values()); - fut0.onDone(failure); + if (U.isLocalNodeCoordinator(ctx.discovery())) + cacheStartProc.start(reqId, reqId); return; } - if (U.isLocalNodeCoordinator(ctx.discovery())) - cacheStartProc.start(reqId, reqId); + // Remove files asynchronously. + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + rollback(opCtx0); + + fut0.onDone(failure); + }); } /** @@ -392,23 +552,39 @@ private IgniteInternalFuture cacheStart(UUID reqId) { GridFutureAdapter retFut = new GridFutureAdapter<>(); - if (log.isInfoEnabled()) { - log.info("Starting restored caches " + - "[snapshot=" + opCtx0.snapshotName() + - ", caches=" + F.viewReadOnly(opCtx0.configs(), c -> c.config().getName()) + ']'); - } + try { + Collection ccfgs = opCtx0.configs(); - ctx.cache().dynamicStartCachesByStoredConf(opCtx.configs(), true, true, false, null, true, opCtx0.nodes()).listen( - f -> { - if (f.error() != null) { - log.error("Unable to start restored caches.", f.error()); + // Ensure that shared cache groups has no conflicts before start caches. + for (StoredCacheData cfg : ccfgs) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - retFut.onDone(f.error()); - } - else - retFut.onDone(true); + if (log.isInfoEnabled()) { + log.info("Starting restored caches " + + "[snapshot=" + opCtx0.snapshotName() + + ", caches=" + F.viewReadOnly(opCtx0.configs(), c -> c.config().getName()) + ']'); } - ); + + ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes()).listen( + f -> { + if (f.error() != null) { + log.error("Unable to start restored caches [groups=" + opCtx0.groups() + + ", snapshot=" + opCtx0.snapshotName() + ']', f.error()); + + retFut.onDone(f.error()); + } + else + retFut.onDone(true); + } + ); + } catch (Exception e) { + log.error("Unable to restore cache group(s) from snapshot " + + "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', e); + + return new GridFinishedFuture<>(e); + } return retFut; } @@ -425,24 +601,38 @@ private void finishCacheStart(UUID reqId, Map res, Map leftNodes = new HashSet<>(opCtx0.nodes()); + if (failure == null) { + fut0.onDone(); - leftNodes.removeAll(res.keySet()); - - failure = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); + return; } - if (failure != null) { - if (U.isLocalNodeCoordinator(ctx.discovery())) - rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); + if (U.isLocalNodeCoordinator(ctx.discovery())) + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); + } - return; + /** + * Check the response for probable failures. + * + * @param errs Errors. + * @param expNodes Expected set of responding topology nodes. + * @param respNodes Set of responding topology nodes. + * @return Error, if any. + */ + private Exception checkFailure(Map errs, Set expNodes, Set respNodes) { + Exception err = F.first(errs.values()); + + if (err == null && !respNodes.containsAll(expNodes)) { + Set leftNodes = new HashSet<>(expNodes); + + leftNodes.removeAll(respNodes); + + err = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); } - fut0.onDone(); + return err; } /** @@ -461,7 +651,7 @@ private IgniteInternalFuture rollback(SnapshotR if (log.isInfoEnabled()) log.info("Performing rollback routine for restored cache groups [groups=" + opCtx0.groups() + ']'); - opCtx0.rollback(); + rollback(opCtx0); return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse(req.error())); } @@ -501,7 +691,7 @@ private boolean allNodesInBaselineAndAlive(Set nodeIds) { /** * Cache group restore from snapshot operation context. */ - private class SnapshotRestoreContext { + private static class SnapshotRestoreContext { /** Request ID. */ private final UUID reqId; @@ -511,43 +701,40 @@ private class SnapshotRestoreContext { /** Baseline node IDs that must be alive to complete the operation. */ private final Set reqNodes; - /** List of processed cache IDs. */ - private final Set cacheIds = new HashSet<>(); + /** List of cache group names to restore from the snapshot. */ + private final Set grps; - /** Cache configurations. */ - private final List ccfgs; + /** Set of processed cache IDs. */ + private final Set cacheIds = new GridConcurrentHashSet<>(); - /** Restored cache groups. */ - private final Map> grps = new ConcurrentHashMap<>(); + /** Directories to clean up if the restore procedure fails. */ + private final Queue grpDirs = new ConcurrentLinkedQueue<>(); /** The exception that led to the interruption of the process. */ private final AtomicReference errRef = new AtomicReference<>(); + /** Collection of cache configurations */ + private volatile List ccfgs; + /** * @param reqId Request ID. * @param snpName Snapshot name. + * @param grps List of cache group names to restore from the snapshot. * @param reqNodes Baseline node IDs that must be alive to complete the operation. - * @param cfgs Stored cache configurations. */ - protected SnapshotRestoreContext(UUID reqId, String snpName, Set reqNodes, List cfgs) { - ccfgs = new ArrayList<>(cfgs); - - for (StoredCacheData cacheData : cfgs) { - String cacheName = cacheData.config().getName(); - - cacheIds.add(CU.cacheId(cacheName)); - - boolean shared = cacheData.config().getGroupName() != null; + protected SnapshotRestoreContext(UUID reqId, String snpName, Collection grps, Set reqNodes) { + this.reqId = reqId; + this.reqNodes = new HashSet<>(reqNodes); + this.snpName = snpName; - grps.computeIfAbsent(shared ? cacheData.config().getGroupName() : cacheName, v -> new ArrayList<>()); + Set grps0 = new HashSet<>(); - if (shared) - cacheIds.add(CU.cacheId(cacheData.config().getGroupName())); + for (String grpName : grps) { + grps0.add(grpName); + cacheIds.add(CU.cacheId(grpName)); } - this.reqId = reqId; - this.reqNodes = new HashSet<>(reqNodes); - this.snpName = snpName; + this.grps = grps0; } /** @return Request ID. */ @@ -569,7 +756,7 @@ protected String snapshotName() { * @return List of cache group names to restore from the snapshot. */ protected Set groups() { - return grps.keySet(); + return grps; } /** @@ -580,7 +767,23 @@ protected boolean containsCache(String name) { return cacheIds.contains(CU.cacheId(name)); } - /** @return Cache configurations. */ + /** + * @param ccfgs Collection of cache configurations. + */ + protected void configs(Collection ccfgs) { + List ccfgs0 = new ArrayList<>(ccfgs.size()); + + for (StoredCacheData cacheData : ccfgs) { + ccfgs0.add(cacheData); + + if (cacheData.config().getGroupName() != null) + cacheIds.add(CU.cacheId(cacheData.config().getName())); + } + + this.ccfgs = ccfgs0; + } + + /** @return Collection of cache configurations */ protected Collection configs() { return ccfgs; } @@ -608,72 +811,17 @@ protected boolean interrupted() { } /** - * Restore specified cache groups from the local snapshot directory. - * - * @param updateMetadata Update binary metadata flag. - * @throws IgniteCheckedException If failed. + * @param cacheDir Directory to clean up if the restore procedure fails */ - protected void restore(boolean updateMetadata) throws IgniteCheckedException { - if (interrupted()) - return; - - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - - if (updateMetadata) { - File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), folderName); - - if (!binDir.exists()) { - throw new IgniteCheckedException("Unable to update cluster metadata from snapshot, " + - "directory doesn't exists [snapshot=" + snpName + ", dir=" + binDir + ']'); - } - - ctx.cacheObjects().updateMetadata(binDir, this::interrupted); - } - - for (File grpDir : snapshotMgr.snapshotCacheDirectories(snpName, folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(grpDir); - - if (!groups().contains(grpName)) - continue; - - snapshotMgr.restoreCacheGroupFiles(snpName, grpName, grpDir, this::interrupted, grps.get(grpName)); - } + protected void pushDir(File cacheDir) { + grpDirs.offer(cacheDir); } /** - * Rollback changes made by process in specified cache group. + * @return Directory to clean up or {@code null} if nothing to clean up. */ - protected void rollback() { - if (groups().isEmpty()) - return; - - List grpNames = new ArrayList<>(groups()); - - for (String grpName : grpNames) { - List files = grps.remove(grpName); - - if (files == null) - continue; - - List dirs = new ArrayList<>(); - - for (File file : files) { - if (!file.exists()) - continue; - - if (file.isDirectory()) - dirs.add(file); - - if (!file.delete()) - log.warning("Unable to delete a file created during a cache restore operation [file=" + file + ']'); - } - - for (File dir : dirs) { - if (!dir.delete()) - log.warning("Unable to delete a folder created during a cache restore operation [file=" + dir + ']'); - } - } + private @Nullable File popDir() { + return grpDirs.poll(); } } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java deleted file mode 100644 index dbb8d75fd00407..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationResult.java +++ /dev/null @@ -1,60 +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.Serializable; -import java.util.List; -import java.util.UUID; -import org.apache.ignite.internal.processors.cache.StoredCacheData; - -/** - * Result of a cache group restore verification job. - */ -public class SnapshotRestoreVerificationResult implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** List of stored cache configurations. */ - private final List ccfgs; - - /** Local node ID. */ - private final UUID locNodeId; - - /** - * @param ccfgs List of stored cache configurations. - * @param locNodeId Local node ID. - */ - public SnapshotRestoreVerificationResult(List ccfgs, UUID locNodeId) { - this.ccfgs = ccfgs; - this.locNodeId = locNodeId; - } - - /** - * @return List of stored cache configurations. - */ - public List configs() { - return ccfgs; - } - - /** - * @return Local node ID. - */ - public UUID localNodeId() { - return locNodeId; - } -} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java deleted file mode 100644 index a8914fe3c56da2..00000000000000 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreVerificationTask.java +++ /dev/null @@ -1,200 +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.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import org.apache.ignite.IgniteCheckedException; -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.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.StoredCacheData; -import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.resources.IgniteInstanceResource; -import org.jetbrains.annotations.NotNull; - -import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; - -/** - * Verification task for restoring a cache group from a snapshot. - */ -public class SnapshotRestoreVerificationTask extends ComputeTaskAdapter>> { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Snapshot name. */ - private final String snpName; - - /** List of cache group names to restore from the snapshot. */ - @GridToStringInclude - private final Collection grps; - - /** - * @param snpName Snapshot name. - * @param grps List of cache group names to restore from the snapshot. - */ - public SnapshotRestoreVerificationTask(String snpName, Collection grps) { - this.snpName = snpName; - this.grps = grps; - } - - /** {@inheritDoc} */ - @Override public @NotNull Map map(List subgrid, - Void arg) throws IgniteException { - Map jobs = new HashMap<>(); - - for (ClusterNode node : subgrid) - jobs.put(new SnapshotRestoreVerificationJob(snpName, grps), node); - - return jobs; - } - - /** {@inheritDoc} */ - @Override public Map> reduce(List results) throws IgniteException { - Map> resMap = new HashMap<>(); - ComputeJobResult firstNodeRes = null; - - for (ComputeJobResult jobRes : results) { - List res = jobRes.getData(); - - if (res == null) - continue; - - resMap.put(jobRes.getNode().id(), res); - - if (firstNodeRes == null) { - firstNodeRes = jobRes; - - continue; - } - - int expCfgCnt = ((Collection)firstNodeRes.getData()).size(); - - if (expCfgCnt != res.size()) { - throw new IgniteException("Count of cache configs mismatch [" + - "node1=" + firstNodeRes.getNode().id() + ", cnt1=" + expCfgCnt + - ", node2=" + jobRes.getNode().id() + ", cnt2=" + res.size() + ']'); - } - } - - return resMap; - } - - /** {@inheritDoc} */ - @Override public ComputeJobResultPolicy result(ComputeJobResult res, List rcvd) { - IgniteException e = res.getException(); - - // Don't failover this job, if topology changed - user should restart operation. - if (e != null) - throw e; - - return super.result(res, rcvd); - } - - /** */ - private static class SnapshotRestoreVerificationJob extends ComputeJobAdapter { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Auto-injected grid instance. */ - @IgniteInstanceResource - private transient IgniteEx ignite; - - /** Snapshot name. */ - private final String snpName; - - /** List of cache group names to restore from the snapshot. */ - @GridToStringInclude - private final Collection grps; - - /** - * @param snpName Snapshot name. - * @param grps List of cache group names to restore from the snapshot. - */ - public SnapshotRestoreVerificationJob(String snpName, Collection grps) { - this.snpName = snpName; - this.grps = grps; - } - - /** {@inheritDoc} */ - @Override public Object execute() throws IgniteException { - assert !ignite.context().clientNode(); - - try { - Map ccfgs = new HashMap<>(); - GridCacheSharedContext cctx = ignite.context().cache().context(); - String folderName = ignite.context().pdsFolderResolver().resolveFolders().folderName(); - - List metas = cctx.snapshotMgr().readSnapshotMetadatas(snpName); - - if (F.isEmpty(metas)) - return null; - - SnapshotMetadata meta = metas.get(0); - - if (!meta.consistentId().equals(cctx.localNode().consistentId().toString())) - return null; - - if (meta.pageSize() != cctx.database().pageSize()) { - throw new IgniteCheckedException("Incompatible memory page size " + - "[snapshotPageSize=" + meta.pageSize() + - ", local=" + cctx.database().pageSize() + - ", snapshot=" + snpName + - ", nodeId=" + cctx.localNodeId() + ']'); - } - - // Collect cache configuration(s) and verify cache groups page size. - for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(snpName, folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(cacheDir); - - if (!grps.contains(grpName)) - continue; - - ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, ccfgs); - } - - if (ccfgs.isEmpty()) - return null; - - File binDir = binaryWorkDir( - cctx.snapshotMgr().snapshotLocalDir(snpName).getAbsolutePath(), - folderName); - - ignite.context().cacheObjects().checkMetadata(binDir); - - return new ArrayList<>(ccfgs.values()); - } - catch (IgniteCheckedException e) { - throw new IgniteException(e); - } - } - } -} 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 e4fda45ace8138..c702dffc14aff8 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 @@ -308,6 +308,8 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { cacheCfg1.setNodeFilter(node -> node.id().equals(nodeId0)); cacheCfg2.setNodeFilter(node -> node.id().equals(nodeId1)); + putKeys(ignite0.cache(dfltCacheCfg.getName()), 0, CACHE_KEYS_RANGE); + IgniteCache cache1 = ignite0.createCache(cacheCfg1); putKeys(cache1, 0, CACHE_KEYS_RANGE); @@ -325,8 +327,6 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { awaitPartitionMapExchange(); - U.sleep(2_000); - // After destroying the cache with a node filter, the configuration file remains on the filtered node. // todo https://issues.apache.org/jira/browse/IGNITE-14044 for (String cacheName : new String[] {cacheName1, cacheName2}) { @@ -457,6 +457,8 @@ private void checkCacheStartWithTheSameName( String grpName = "shared"; String cacheName = "cache1"; + boolean prepare = procType == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; + dfltCacheCfg = txCacheConfig(new CacheConfiguration(cacheName)).setGroupName(grpName); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); @@ -466,13 +468,25 @@ private void checkCacheStartWithTheSameName( IgniteFuture fut = waitForBlockOnRestore(spi, procType, grpName); GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(grpName), IgniteCheckedException.class, null); - GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); + + if (!prepare) + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); + else + ignite.createCache(cacheName); spi.stopBlock(); - fut.get(TIMEOUT); + if (prepare) { + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), IllegalStateException.class, + "Cache \"cache1\" should be destroyed manually before perform restore operation."); - checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); + ensureCacheDirEmpty(2, grpName); + } + else { + fut.get(TIMEOUT); + + checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); + } } /** @throws Exception If failed. */ From 3b6188a8c2f7b4e60fcf050efb94d8a65e9c6039 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 10 Mar 2021 22:50:25 +0300 Subject: [PATCH 36/98] IGNITE-13805 Review note. --- .../GridCacheDatabaseSharedManager.java | 2 +- .../SnapshotRestorePrepareRequest.java | 20 ++-- .../snapshot/SnapshotRestoreProcess.java | 106 ++++++++++-------- .../IgniteClusterSnapshotRestoreSelfTest.java | 6 +- 4 files changed, 71 insertions(+), 63 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java index bb6950102f4b46..e6ca08c968ef01 100755 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java @@ -1357,7 +1357,7 @@ else if (regCfg.getPageEvictionMode() != DataPageEvictionMode.DISABLED) { /** * @param partFile Partition file. */ - public int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException { + private int resolvePageSizeFromPartitionFile(Path partFile) throws IOException, IgniteCheckedException { FileIOFactory ioFactory = persistenceCfg.getFileIOFactory(); try (FileIO fileIO = ioFactory.create(partFile.toFile())) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java index cd7df1dbd6137b..0acc3b404aeb07 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java @@ -21,7 +21,6 @@ import java.util.Collection; import java.util.Set; import java.util.UUID; -import org.apache.ignite.internal.util.tostring.GridToStringExclude; import org.apache.ignite.internal.util.typedef.internal.S; /** @@ -40,32 +39,31 @@ public class SnapshotRestorePrepareRequest implements Serializable { /** Baseline node IDs that must be alive to complete the operation. */ private final Set nodes; - /** Stored cache configurations. */ - @GridToStringExclude + /** List of cache group names to restore from the snapshot. */ private final Collection grps; /** Node ID from which to update the binary metadata. */ - private final UUID performNodeId; + private final UUID updateMetaNodeId; /** * @param reqId Request ID. * @param snpName Snapshot name. * @param nodes Baseline node IDs that must be alive to complete the operation. - * @param ccfgs Stored cache configurations. - * @param performNodeId Node ID from which to update the binary metadata. + * @param grps List of cache group names to restore from the snapshot. + * @param updateMetaNodeId Node ID from which to update the binary metadata. */ public SnapshotRestorePrepareRequest( UUID reqId, String snpName, Set nodes, Collection grps, - UUID performNodeId + UUID updateMetaNodeId ) { this.reqId = reqId; this.snpName = snpName; this.nodes = nodes; this.grps = grps; - this.performNodeId = performNodeId; + this.updateMetaNodeId = updateMetaNodeId; } /** @@ -83,7 +81,7 @@ public String snapshotName() { } /** - * @return Stored cache configurations. + * @return List of cache group names to restore from the snapshot. */ public Collection groups() { return grps; @@ -99,8 +97,8 @@ public Set nodes() { /** * @return Node ID from which to update the binary metadata. */ - public UUID performNodeId() { - return performNodeId; + public UUID updateMetaNodeId() { + return updateMetaNodeId; } /** {@inheritDoc} */ 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 f52617cc0e8db0..07f9d30056a40d 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 @@ -92,7 +92,7 @@ public class SnapshotRestoreProcess { private final IgniteLogger log; /** The future to be completed when the cache restore process is complete. */ - private volatile GridFutureAdapter fut = new GridFutureAdapter<>(); + private volatile GridFutureAdapter fut; /** Snapshot restore operation context. */ private volatile SnapshotRestoreContext opCtx; @@ -113,8 +113,6 @@ public SnapshotRestoreProcess(GridKernalContext ctx) { rollbackRestoreProc = new DistributedProcess<>( ctx, RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK, this::rollback, this::finishRollback); - - fut.onDone(); } /** @@ -131,9 +129,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } synchronized (this) { - IgniteInternalFuture fut0 = fut; + GridFutureAdapter fut0 = fut; - if (!fut0.isDone()) { + if (opCtx != null || (fut0 != null && !fut0.isDone())) { return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); } @@ -235,16 +233,39 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames public boolean inProgress(@Nullable String cacheName) { SnapshotRestoreContext opCtx0 = opCtx; - return !staleProcess(fut, opCtx0) && (cacheName == null || opCtx0.containsCache(cacheName)); + return opCtx0 != null && (cacheName == null || opCtx0.containsCache(cacheName)); + } + + /** + * Finish local cache group restore process. + */ + private void finishProcess() { + finishProcess(null); } /** - * @param fut The future of cache snapshot restore operation. - * @param opCtx Snapshot restore operation context. - * @return {@code True} if the future completed or not initiated. + * Finish local cache group restore process. + * + * @param err Error, if any. */ - public boolean staleProcess(IgniteInternalFuture fut, SnapshotRestoreContext opCtx) { - return fut.isDone() || opCtx == null; + private void finishProcess(@Nullable Throwable err) { + SnapshotRestoreContext opCtx0 = opCtx; + + if (err != null) { + log.error("Failed to restore snapshot cache group [" + + "groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', err); + } + else if (log.isInfoEnabled()) { + log.info("Successfully restored cache group(s) from the snapshot [" + + "group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']'); + } + + GridFutureAdapter fut0 = fut; + + if (fut0 != null) + fut0.onDone(null, err); + + opCtx = null; } /** @@ -305,14 +326,8 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (state.state() != ClusterState.ACTIVE || state.transition()) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); - // Skip creating future on initiator. - if (fut.isDone()) - fut = new GridFutureAdapter<>(); - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.groups(), req.nodes()); - fut.listen(f -> opCtx = null); - try { Map cfgMap = new HashMap<>(); GridCacheSharedContext cctx = ctx.cache().context(); @@ -343,11 +358,11 @@ private IgniteInternalFuture> prepare(SnapshotRestore ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cfgMap); } - ArrayList ccfgs = new ArrayList<>(cfgMap.values()); - - if (ccfgs.isEmpty()) + if (cfgMap.isEmpty()) return new GridFinishedFuture<>(); + ArrayList ccfgs = new ArrayList<>(cfgMap.values()); + if (!allNodesInBaselineAndAlive(req.nodes())) throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); @@ -360,9 +375,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore cctx.snapshotMgr().snapshotExecutorService().execute(() -> { try { - ctx.cacheObjects().checkMetadata(binDir); - - restore(opCtx0, binDir, ctx.localNodeId().equals(req.performNodeId())); + restore(opCtx0, binDir, ctx.localNodeId().equals(req.updateMetaNodeId())); if (!opCtx0.interrupted()) { retFut.onDone(ccfgs); @@ -402,15 +415,17 @@ protected void restore(SnapshotRestoreContext opCtx, File binDir, boolean update if (opCtx.interrupted()) return; - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - if (updateMetadata) { - assert binDir.exists(); + // Check binary metadata compatibility. + ctx.cacheObjects().checkMetadata(binDir); + // Cluster-wide update binary metadata. ctx.cacheObjects().updateMetadata(binDir, opCtx::interrupted); } + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); + for (File snpCacheDir : snapshotMgr.snapshotCacheDirectories(opCtx.snapshotName(), folderName)) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); @@ -430,7 +445,7 @@ protected void restore(SnapshotRestoreContext opCtx, File binDir, boolean update opCtx.pushDir(cacheDir); - restoreCacheGroupFiles(opCtx.snapshotName(), grpName, cacheDir, snpCacheDir, opCtx::interrupted); + copyPartitions(opCtx.snapshotName(), grpName, cacheDir, snpCacheDir, opCtx::interrupted); } } @@ -442,7 +457,7 @@ protected void restore(SnapshotRestoreContext opCtx, File binDir, boolean update * @param stopChecker Node stop or prcoess interrupt checker. * @throws IgniteCheckedException If failed. */ - protected void restoreCacheGroupFiles(String snpName, String grpName, File cacheDir, File snpCacheDir, + protected void copyPartitions(String snpName, String grpName, File cacheDir, File snpCacheDir, BooleanSupplier stopChecker) throws IgniteCheckedException { try { if (log.isInfoEnabled()) @@ -488,10 +503,9 @@ protected void rollback(SnapshotRestoreContext opCtx) { * @param errs Errors. */ private void finishPrepare(UUID reqId, Map> res, Map errs) { - GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut0, opCtx0)) + if (opCtx0 == null) return; Exception failure = checkFailure(errs, opCtx0.nodes(), res.keySet()); @@ -507,7 +521,7 @@ private void finishPrepare(UUID reqId, Map> res filteredCfgs.put(cacheData.config().getName(), cacheData); } - opCtx.configs(filteredCfgs.values()); + opCtx0.configs(filteredCfgs.values()); if (U.isLocalNodeCoordinator(ctx.discovery())) cacheStartProc.start(reqId, reqId); @@ -519,7 +533,7 @@ private void finishPrepare(UUID reqId, Map> res ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { rollback(opCtx0); - fut0.onDone(failure); + finishProcess(failure); }); } @@ -530,7 +544,7 @@ private void finishPrepare(UUID reqId, Map> res private IgniteInternalFuture cacheStart(UUID reqId) { SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut, opCtx0)) + if (opCtx0 == null) return new GridFinishedFuture<>(); if (!reqId.equals(opCtx0.requestId())) @@ -595,16 +609,15 @@ private IgniteInternalFuture cacheStart(UUID reqId) { * @param errs Errors. */ private void finishCacheStart(UUID reqId, Map res, Map errs) { - GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut0, opCtx0) || !reqId.equals(opCtx0.requestId())) + if (opCtx0 == null || !reqId.equals(opCtx0.requestId())) return; Exception failure = checkFailure(errs, opCtx0.nodes(), res.keySet()); if (failure == null) { - fut0.onDone(); + finishProcess(); return; } @@ -642,14 +655,14 @@ private Exception checkFailure(Map errs, Set expNodes, Se private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut, opCtx0) || !req.requestId().equals(opCtx0.requestId())) + if (opCtx0 == null || !req.requestId().equals(opCtx0.requestId())) return new GridFinishedFuture<>(); if (!opCtx0.nodes().contains(ctx.localNodeId())) return new GridFinishedFuture<>(); if (log.isInfoEnabled()) - log.info("Performing rollback routine for restored cache groups [groups=" + opCtx0.groups() + ']'); + log.info("Performing rollback routine for restored cache group(s) [groups=" + opCtx0.groups() + ']'); rollback(opCtx0); @@ -662,15 +675,14 @@ private IgniteInternalFuture rollback(SnapshotR * @param errs Errors. */ private void finishRollback(UUID reqId, Map res, Map errs) { - GridFutureAdapter fut0 = fut; SnapshotRestoreContext opCtx0 = opCtx; - if (staleProcess(fut0, opCtx0) || !reqId.equals(opCtx0.requestId())) + if (opCtx0 == null || !reqId.equals(opCtx0.requestId())) return; SnapshotRestoreRollbackResponse resp = F.first(F.viewReadOnly(res.values(), v -> v, Objects::nonNull)); - fut0.onDone(resp.error()); + finishProcess(resp.error()); } /** @@ -699,7 +711,7 @@ private static class SnapshotRestoreContext { private final String snpName; /** Baseline node IDs that must be alive to complete the operation. */ - private final Set reqNodes; + private final Set nodes; /** List of cache group names to restore from the snapshot. */ private final Set grps; @@ -720,11 +732,11 @@ private static class SnapshotRestoreContext { * @param reqId Request ID. * @param snpName Snapshot name. * @param grps List of cache group names to restore from the snapshot. - * @param reqNodes Baseline node IDs that must be alive to complete the operation. + * @param nodes Baseline node IDs that must be alive to complete the operation. */ - protected SnapshotRestoreContext(UUID reqId, String snpName, Collection grps, Set reqNodes) { + protected SnapshotRestoreContext(UUID reqId, String snpName, Collection grps, Collection nodes) { this.reqId = reqId; - this.reqNodes = new HashSet<>(reqNodes); + this.nodes = new HashSet<>(nodes); this.snpName = snpName; Set grps0 = new HashSet<>(); @@ -744,7 +756,7 @@ protected UUID requestId() { /** @return Baseline node IDs that must be alive to complete the operation. */ protected Set nodes() { - return Collections.unmodifiableSet(reqNodes); + return Collections.unmodifiableSet(nodes); } /** @return Snapshot name. */ 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 c702dffc14aff8..df8b6ecdb9149c 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 @@ -321,8 +321,6 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { cache1.destroy(); cache2.destroy(); - awaitPartitionMapExchange(); - forceCheckpoint(); awaitPartitionMapExchange(); @@ -334,10 +332,10 @@ public void testRestoreCacheGroupWithNodeFilter() throws Exception { U.delete(resolveCacheDir(grid(nodeIdx), cacheName)); } - ignite0.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); + ignite1.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); awaitPartitionMapExchange(); - ignite1.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); + ignite0.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); awaitPartitionMapExchange(); checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); From a72283649a07d5587ec753a25918ba162be77493 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 12 Mar 2021 13:29:10 +0300 Subject: [PATCH 37/98] IGNITE-13805 Don't store cache dirs in context. --- .../snapshot/SnapshotRestoreProcess.java | 239 ++++++++---------- .../IgniteClusterSnapshotRestoreSelfTest.java | 19 +- 2 files changed, 115 insertions(+), 143 deletions(-) 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 07f9d30056a40d..7476774ef10299 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 @@ -30,12 +30,9 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Queue; import java.util.Set; import java.util.UUID; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -57,6 +54,7 @@ import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; @@ -252,12 +250,12 @@ private void finishProcess(@Nullable Throwable err) { SnapshotRestoreContext opCtx0 = opCtx; if (err != null) { - log.error("Failed to restore snapshot cache group [" + - "groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', err); + log.error("Failed to restore snapshot cache group" + (opCtx0 == null ? "" : + " [group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']'), err); } else if (log.isInfoEnabled()) { - log.info("Successfully restored cache group(s) from the snapshot [" + - "group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']'); + log.info("Successfully restored cache group(s) from the snapshot" + (opCtx0 == null ? "" : + " [group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']')); } GridFutureAdapter fut0 = fut; @@ -326,19 +324,21 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (state.state() != ClusterState.ACTIVE || state.transition()) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.groups(), req.nodes()); - try { + if (!allNodesInBaselineAndAlive(req.nodes())) + throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + + for (String grpName : req.groups()) + ensureCacheAbsent(grpName); + Map cfgMap = new HashMap<>(); GridCacheSharedContext cctx = ctx.cache().context(); String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); - if (meta == null || !meta.consistentId().equals(cctx.localNode().consistentId().toString())) - return new GridFinishedFuture<>(); - - if (meta.pageSize() != cctx.database().pageSize()) { + if (meta != null && meta.consistentId().equals(cctx.localNode().consistentId().toString()) + && meta.pageSize() != cctx.database().pageSize()) { throw new IgniteCheckedException("Incompatible memory page size " + "[snapshotPageSize=" + meta.pageSize() + ", local=" + cctx.database().pageSize() + @@ -346,28 +346,38 @@ private IgniteInternalFuture> prepare(SnapshotRestore ", nodeId=" + cctx.localNodeId() + ']'); } - SnapshotRestoreContext opCtx0 = opCtx; + List> cacheDirs = new ArrayList<>(); // Collect cache configuration(s) and verify cache groups page size. - for (File cacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(cacheDir); + for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), folderName)) { + String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); - if (!opCtx0.groups().contains(grpName)) + if (!req.groups().contains(grpName)) continue; - ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(cacheDir, cfgMap); + ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(snpCacheDir, cfgMap); + + File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), + Paths.get(databaseRelativePath(folderName), snpCacheDir.getName()).toString(), false); + + if (!cacheDir.exists()) + cacheDir.mkdir(); + else if (cacheDir.list().length > 0) { + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } + + cacheDirs.add(new T2<>(snpCacheDir, cacheDir)); } + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.groups(), req.nodes()); + if (cfgMap.isEmpty()) return new GridFinishedFuture<>(); - ArrayList ccfgs = new ArrayList<>(cfgMap.values()); - - if (!allNodesInBaselineAndAlive(req.nodes())) - throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + SnapshotRestoreContext opCtx0 = opCtx; - for (String grpName : opCtx0.groups()) - ensureCacheAbsent(grpName); + opCtx0.configs(new ArrayList<>(cfgMap.values())); File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(req.snapshotName()).getAbsolutePath(), folderName); @@ -375,10 +385,21 @@ private IgniteInternalFuture> prepare(SnapshotRestore cctx.snapshotMgr().snapshotExecutorService().execute(() -> { try { - restore(opCtx0, binDir, ctx.localNodeId().equals(req.updateMetaNodeId())); + if (opCtx0.interrupted()) + return; + + if (ctx.localNodeId().equals(req.updateMetaNodeId())) { + // Check binary metadata compatibility. + ctx.cacheObjects().checkMetadata(binDir); + + // Cluster-wide update binary metadata. + ctx.cacheObjects().updateMetadata(binDir, opCtx0::interrupted); + } + + copyPartitions(cacheDirs, opCtx0); if (!opCtx0.interrupted()) { - retFut.onDone(ccfgs); + retFut.onDone(new ArrayList<>(opCtx0.configs())); return; } @@ -398,102 +419,79 @@ private IgniteInternalFuture> prepare(SnapshotRestore return retFut; } catch (Exception e) { - log.error("Unable to restore cache group(s) from snapshot " + - "[groups=" + req.groups() + ", snapshot=" + req.snapshotName() + ']', e); - return new GridFinishedFuture<>(e); } } /** - * Restore specified cache groups from the local snapshot directory. + * Copy partition filess from the local snapshot directory. * - * @param updateMetadata Update binary metadata flag. + * @param dirs Cache directories. + * @param opCtx Snapshot restore operation context. * @throws IgniteCheckedException If failed. */ - protected void restore(SnapshotRestoreContext opCtx, File binDir, boolean updateMetadata) throws IgniteCheckedException { - if (opCtx.interrupted()) - return; - - if (updateMetadata) { - // Check binary metadata compatibility. - ctx.cacheObjects().checkMetadata(binDir); - - // Cluster-wide update binary metadata. - ctx.cacheObjects().updateMetadata(binDir, opCtx::interrupted); - } - - IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - - for (File snpCacheDir : snapshotMgr.snapshotCacheDirectories(opCtx.snapshotName(), folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); + protected void copyPartitions(Collection> dirs, + SnapshotRestoreContext opCtx) throws IgniteCheckedException { + for (T2 cacheDirs : dirs) { + File snpCacheDir = cacheDirs.get1(); + File cacheDir = cacheDirs.get2(); + + try { + if (log.isInfoEnabled()) + log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); + + for (File snpFile : snpCacheDir.listFiles()) { + if (opCtx.interrupted()) + return; - if (!opCtx.groups().contains(grpName)) - continue; + File target = new File(cacheDir, snpFile.getName()); - File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), - Paths.get(databaseRelativePath(folderName), snpCacheDir.getName()).toString(), false); + if (log.isDebugEnabled()) { + log.debug("Copying file from the snapshot " + + "[snapshot=" + opCtx.snapshotName() + + ", src=" + snpFile + + ", target=" + target + "]"); + } - if (!cacheDir.exists()) - cacheDir.mkdir(); - else - if (cacheDir.list().length > 0) { - throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + - "[group=" + grpName + ", dir=" + cacheDir + ']'); + Files.copy(snpFile.toPath(), target.toPath()); + } + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snapshotName() + + ", grp=" + FilePageStoreManager.cacheGroupName(cacheDir) + ']', e); } - - opCtx.pushDir(cacheDir); - - copyPartitions(opCtx.snapshotName(), grpName, cacheDir, snpCacheDir, opCtx::interrupted); } } /** - * @param snpName Snapshot name. - * @param grpName Cache group name. - * @param cacheDir Cache group directory. - * @param snpCacheDir Cache group directory in snapshot. - * @param stopChecker Node stop or prcoess interrupt checker. - * @throws IgniteCheckedException If failed. + * Rollback changes made by process in specified cache group. + * + * @param opCtx Snapshot restore operation context. */ - protected void copyPartitions(String snpName, String grpName, File cacheDir, File snpCacheDir, - BooleanSupplier stopChecker) throws IgniteCheckedException { + private void rollback(@Nullable SnapshotRestoreContext opCtx) { + if (opCtx == null) + return; + try { - if (log.isInfoEnabled()) - log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); + for (StoredCacheData cacheData : opCtx.configs()) { + boolean shared = !F.isEmpty(cacheData.config().getGroupName()); - for (File snpFile : snpCacheDir.listFiles()) { - if (stopChecker.getAsBoolean()) - return; + String grpName = shared ? cacheData.config().getGroupName() : cacheData.config().getName(); - File target = new File(cacheDir, snpFile.getName()); + File cacheDir = ((FilePageStoreManager)ctx.cache().context().pageStore()).cacheWorkDir(shared, grpName); - if (log.isDebugEnabled()) { - log.debug("Copying file from the snapshot " + - "[snapshot=" + snpName + - ", grp=" + grpName + - ", src=" + snpFile + - ", target=" + target + "]"); - } + if (!cacheDir.exists()) + continue; + + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); - Files.copy(snpFile.toPath(), target.toPath()); + U.delete(cacheDir); } } - catch (IOException e) { - throw new IgniteCheckedException("Unable to copy file [snapshot=" + snpName + ", grp=" + grpName + ']', e); - } - } - - /** - * Rollback changes made by process in specified cache group. - */ - protected void rollback(SnapshotRestoreContext opCtx) { - File rmvDir; - - while ((rmvDir = opCtx.popDir()) != null) { - if (!U.delete(rmvDir)) - log.error("Unable to delete restored cache directory [dir=" + rmvDir + ']'); + catch (Exception e) { + log.error("Failed to perform rollback [group(s)=" + opCtx.groups() + + ", snapshot=" + opCtx.snapshotName() + ']', e); } } @@ -505,12 +503,14 @@ protected void rollback(SnapshotRestoreContext opCtx) { private void finishPrepare(UUID reqId, Map> res, Map errs) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 == null) + if (ctx.clientNode()) return; - Exception failure = checkFailure(errs, opCtx0.nodes(), res.keySet()); + Exception failure = checkFailure(errs, opCtx0, res.keySet()); if (failure == null) { + assert opCtx0 != null : ctx.localNodeId(); + Map filteredCfgs = new HashMap<>(); for (List storedCfgs : res.values()) { @@ -529,12 +529,14 @@ private void finishPrepare(UUID reqId, Map> res return; } - // Remove files asynchronously. - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - rollback(opCtx0); - + if (opCtx0 == null) finishProcess(failure); - }); + else // Remove files asynchronously. + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + rollback(opCtx0); + + finishProcess(failure); + }); } /** @@ -614,7 +616,7 @@ private void finishCacheStart(UUID reqId, Map res, Map res, Map errs, Set expNodes, Set respNodes) { + private Exception checkFailure(Map errs, SnapshotRestoreContext opCtx, Set respNodes) { Exception err = F.first(errs.values()); - if (err == null && !respNodes.containsAll(expNodes)) { - Set leftNodes = new HashSet<>(expNodes); + if (err == null && opCtx != null && !respNodes.containsAll(opCtx.nodes())) { + Set leftNodes = new HashSet<>(opCtx.nodes()); leftNodes.removeAll(respNodes); @@ -719,9 +721,6 @@ private static class SnapshotRestoreContext { /** Set of processed cache IDs. */ private final Set cacheIds = new GridConcurrentHashSet<>(); - /** Directories to clean up if the restore procedure fails. */ - private final Queue grpDirs = new ConcurrentLinkedQueue<>(); - /** The exception that led to the interruption of the process. */ private final AtomicReference errRef = new AtomicReference<>(); @@ -821,19 +820,5 @@ protected boolean interrupted() { protected @Nullable Throwable error() { return errRef.get(); } - - /** - * @param cacheDir Directory to clean up if the restore procedure fails - */ - protected void pushDir(File cacheDir) { - grpDirs.offer(cacheDir); - } - - /** - * @return Directory to clean up or {@code null} if nothing to clean up. - */ - private @Nullable File popDir() { - return grpDirs.poll(); - } } } 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 df8b6ecdb9149c..8c65e0fe07b983 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 @@ -455,8 +455,6 @@ private void checkCacheStartWithTheSameName( String grpName = "shared"; String cacheName = "cache1"; - boolean prepare = procType == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE; - dfltCacheCfg = txCacheConfig(new CacheConfiguration(cacheName)).setGroupName(grpName); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); @@ -467,24 +465,13 @@ private void checkCacheStartWithTheSameName( GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(grpName), IgniteCheckedException.class, null); - if (!prepare) - GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); - else - ignite.createCache(cacheName); + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); spi.stopBlock(); - if (prepare) { - GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), IllegalStateException.class, - "Cache \"cache1\" should be destroyed manually before perform restore operation."); - - ensureCacheDirEmpty(2, grpName); - } - else { - fut.get(TIMEOUT); + fut.get(TIMEOUT); - checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); - } + checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ From 18337b38fda7ef09fe97aefb9e982576f3c25980 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 15 Mar 2021 20:57:51 +0300 Subject: [PATCH 38/98] IGNITE-13805 Simplify restore context. --- .../processors/cache/ClusterCachesInfo.java | 10 +- .../processors/cache/GridCacheProcessor.java | 2 +- .../snapshot/IgniteSnapshotManager.java | 22 +- .../snapshot/SnapshotRestoreProcess.java | 291 ++++++++---------- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 +- 5 files changed, 148 insertions(+), 181 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index d51cdf06ddfd0a..ead0c55481b1eb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1030,12 +1030,10 @@ else if (encMgr.masterKeyDigest() != null && if (err == null && !req.restoredCache()) { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - String conflictingName = cacheName; - - if (snapshotMgr.isCacheRestoring(conflictingName) || - ((conflictingName = ccfg.getGroupName()) != null && snapshotMgr.isCacheRestoring(conflictingName))) { - err = new IgniteCheckedException("Cache start failed. A cache named \"" + conflictingName + - "\" is currently being restored from a snapshot."); + if (snapshotMgr.isCacheRestoring(cacheName, ccfg.getGroupName())) { + err = new IgniteCheckedException("Cache start failed. A cache or group with the same name is " + + "currently being restored from a snapshot [cache=" + cacheName + + (ccfg.getGroupName() == null ? "" : ", group=" + ccfg.getGroupName()) + ']'); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 087ab4b753496f..5d48a9c91c0caf 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -4346,7 +4346,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { * @return Node validation result if there was an issue with the joining node, {@code null} otherwise. */ private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { - if (ctx.cache().context().snapshotMgr().isCacheRestoring(null)) { + if (ctx.cache().context().snapshotMgr().isSnapshotRestoring()) { String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; return new IgniteNodeValidationResult(node.id(), msg); 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 3b60d375e818ed..fb56874e175827 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 @@ -768,11 +768,23 @@ public boolean isSnapshotCreating() { } /** - * @param name Cache name or {@code null} to check if any cache is currently being restored. - * @return {@code True} if the restore operation is in progress. + * Check if snapshot restore process is currently running. + * + * @return {@code True} if the snapshot restore operation is in progress. */ - public boolean isCacheRestoring(@Nullable String name) { - return restoreCacheGrpProc.inProgress(name); + public boolean isSnapshotRestoring() { + return restoreCacheGrpProc.isSnapshotRestoring(); + } + + /** + * Check if the cache or group with the specified name is currently being restored from the snapshot. + * + * @param cacheName Cache name. + * @param grpName Cache group name. + * @return {@code True} if the cache or group with the specified name is being restored. + */ + public boolean isCacheRestoring(String cacheName, @Nullable String grpName) { + return restoreCacheGrpProc.isCacheRestoring(cacheName, grpName); } /** @@ -1030,7 +1042,7 @@ public List readSnapshotMetadatas(String snpName) { if (!clusterState.hasBaselineTopology()) throw new IgniteException("Snapshot operation has been rejected. The baseline topology is not configured for cluster."); - if (isCacheRestoring(null)) { + if (isSnapshotRestoring()) { throw new IgniteException("Snapshot operation has been rejected. " + "Cache group restore operation is currently in progress."); } 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 7476774ef10299..11193d24adba2f 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 @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.IdentityHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -47,14 +48,12 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; -import org.apache.ignite.internal.util.GridConcurrentHashSet; 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.future.IgniteFinishedFutureImpl; import org.apache.ignite.internal.util.future.IgniteFutureImpl; import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; @@ -224,14 +223,51 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } /** - * Check if the cache group restore process is currently running. + * Check if snapshot restore process is currently running. * - * @return {@code True} if cache group restore process is currently running. + * @return {@code True} if the snapshot restore operation is in progress. */ - public boolean inProgress(@Nullable String cacheName) { + public boolean isSnapshotRestoring() { + return opCtx != null; + } + + /** + * Check if the cache or group with the specified name is currently being restored from the snapshot. + * + * @param cacheName Cache name. + * @param grpName Cache group name. + * @return {@code True} if the cache or group with the specified name is currently being restored. + */ + public boolean isCacheRestoring(String cacheName, @Nullable String grpName) { SnapshotRestoreContext opCtx0 = opCtx; - return opCtx0 != null && (cacheName == null || opCtx0.containsCache(cacheName)); + if (opCtx0 == null) + return false; + + Map cacheCfgs = opCtx0.cfgs; + + int cacheId = CU.cacheId(cacheName); + + if (cacheCfgs.containsKey(cacheId)) + return true; + + for (File grpDir : opCtx0.dirs) { + String locGrpName = FilePageStoreManager.cacheGroupName(grpDir); + + if (grpName == null) { + if (CU.cacheId(locGrpName) == cacheId) + return true; + } + else { + if (cacheName.equals(locGrpName)) + return true; + + if (CU.cacheId(locGrpName) == CU.cacheId(grpName)) + return true; + } + } + + return false; } /** @@ -251,11 +287,11 @@ private void finishProcess(@Nullable Throwable err) { if (err != null) { log.error("Failed to restore snapshot cache group" + (opCtx0 == null ? "" : - " [group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']'), err); + " [requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'), err); } else if (log.isInfoEnabled()) { log.info("Successfully restored cache group(s) from the snapshot" + (opCtx0 == null ? "" : - " [group(s)=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']')); + " [requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']')); } GridFutureAdapter fut0 = fut; @@ -274,8 +310,8 @@ else if (log.isInfoEnabled()) { public void onNodeLeft(UUID leftNodeId) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 != null && opCtx0.nodes().contains(leftNodeId)) { - opCtx0.interrupt(new IgniteException(OP_REJECT_MSG + + if (opCtx0 != null && opCtx0.nodes.contains(leftNodeId)) { + opCtx0.err.compareAndSet(null, new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -289,7 +325,7 @@ public void stop(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 != null) - opCtx0.interrupt(reason); + opCtx0.err.compareAndSet(null, reason); } /** @@ -314,26 +350,22 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (ctx.clientNode()) return new GridFinishedFuture<>(); - if (inProgress(null)) { - return new GridFinishedFuture<>( - new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); - } + try { + if (isSnapshotRestoring()) + throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - DiscoveryDataClusterState state = ctx.state().clusterState(); + DiscoveryDataClusterState state = ctx.state().clusterState(); - if (state.state() != ClusterState.ACTIVE || state.transition()) - return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); + if (state.state() != ClusterState.ACTIVE || state.transition()) + throw new IgniteException(OP_REJECT_MSG + "The cluster should be active."); - try { if (!allNodesInBaselineAndAlive(req.nodes())) throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); for (String grpName : req.groups()) ensureCacheAbsent(grpName); - Map cfgMap = new HashMap<>(); GridCacheSharedContext cctx = ctx.cache().context(); - String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); @@ -346,7 +378,9 @@ private IgniteInternalFuture> prepare(SnapshotRestore ", nodeId=" + cctx.localNodeId() + ']'); } - List> cacheDirs = new ArrayList<>(); + String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); + Map dirs = new IdentityHashMap<>(); + Map cfgMap = new HashMap<>(); // Collect cache configuration(s) and verify cache groups page size. for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), folderName)) { @@ -367,17 +401,24 @@ else if (cacheDir.list().length > 0) { "[group=" + grpName + ", dir=" + cacheDir + ']'); } - cacheDirs.add(new T2<>(snpCacheDir, cacheDir)); + dirs.put(snpCacheDir, cacheDir); } - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.groups(), req.nodes()); + Map idsCfgMap = cfgMap.isEmpty() ? Collections.emptyMap() : + cfgMap.values().stream().collect(Collectors.toMap(v -> CU.cacheId(v.config().getName()), v -> v)); - if (cfgMap.isEmpty()) - return new GridFinishedFuture<>(); + opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), + new ArrayList<>(dirs.values()), idsCfgMap); SnapshotRestoreContext opCtx0 = opCtx; - opCtx0.configs(new ArrayList<>(cfgMap.values())); + if (opCtx0.dirs.isEmpty()) + return new GridFinishedFuture<>(); + + if (log.isInfoEnabled()) { + log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + } File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(req.snapshotName()).getAbsolutePath(), folderName); @@ -385,7 +426,7 @@ else if (cacheDir.list().length > 0) { cctx.snapshotMgr().snapshotExecutorService().execute(() -> { try { - if (opCtx0.interrupted()) + if (opCtx0.err.get() != null) return; if (ctx.localNodeId().equals(req.updateMetaNodeId())) { @@ -393,23 +434,25 @@ else if (cacheDir.list().length > 0) { ctx.cacheObjects().checkMetadata(binDir); // Cluster-wide update binary metadata. - ctx.cacheObjects().updateMetadata(binDir, opCtx0::interrupted); + ctx.cacheObjects().updateMetadata(binDir, () -> opCtx0.err.get() != null); } - copyPartitions(cacheDirs, opCtx0); + copyPartitions(dirs, opCtx0); + + Throwable err = opCtx0.err.get(); - if (!opCtx0.interrupted()) { - retFut.onDone(new ArrayList<>(opCtx0.configs())); + if (err == null) { + retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); return; } log.error("Snapshot restore process has been interrupted " + - "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', opCtx0.error()); + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); rollback(opCtx0); - retFut.onDone(opCtx0.error()); + retFut.onDone(err); } catch (Throwable t) { @@ -430,25 +473,25 @@ else if (cacheDir.list().length > 0) { * @param opCtx Snapshot restore operation context. * @throws IgniteCheckedException If failed. */ - protected void copyPartitions(Collection> dirs, + protected void copyPartitions(Map dirs, SnapshotRestoreContext opCtx) throws IgniteCheckedException { - for (T2 cacheDirs : dirs) { - File snpCacheDir = cacheDirs.get1(); - File cacheDir = cacheDirs.get2(); + for (Map.Entry cacheDirs : dirs.entrySet()) { + File snpCacheDir = cacheDirs.getKey(); + File cacheDir = cacheDirs.getValue(); try { if (log.isInfoEnabled()) log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); for (File snpFile : snpCacheDir.listFiles()) { - if (opCtx.interrupted()) + if (opCtx.err.get() != null) return; File target = new File(cacheDir, snpFile.getName()); if (log.isDebugEnabled()) { log.debug("Copying file from the snapshot " + - "[snapshot=" + opCtx.snapshotName() + + "[snapshot=" + opCtx.snpName + ", src=" + snpFile + ", target=" + target + "]"); } @@ -457,7 +500,7 @@ protected void copyPartitions(Collection> dirs, } } catch (IOException e) { - throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snapshotName() + + throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snpName + ", grp=" + FilePageStoreManager.cacheGroupName(cacheDir) + ']', e); } } @@ -469,17 +512,14 @@ protected void copyPartitions(Collection> dirs, * @param opCtx Snapshot restore operation context. */ private void rollback(@Nullable SnapshotRestoreContext opCtx) { - if (opCtx == null) + if (opCtx == null || F.isEmpty(opCtx.dirs)) return; - try { - for (StoredCacheData cacheData : opCtx.configs()) { - boolean shared = !F.isEmpty(cacheData.config().getGroupName()); - - String grpName = shared ? cacheData.config().getGroupName() : cacheData.config().getName(); - - File cacheDir = ((FilePageStoreManager)ctx.cache().context().pageStore()).cacheWorkDir(shared, grpName); + if (log.isInfoEnabled()) + log.info("Performing local rollback routine for restored cache groups [requestID=" + opCtx.reqId + ']'); + try { + for (File cacheDir : opCtx.dirs) { if (!cacheDir.exists()) continue; @@ -490,8 +530,7 @@ private void rollback(@Nullable SnapshotRestoreContext opCtx) { } } catch (Exception e) { - log.error("Failed to perform rollback [group(s)=" + opCtx.groups() + - ", snapshot=" + opCtx.snapshotName() + ']', e); + log.error("Failed to perform rollback [requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', e); } } @@ -511,17 +550,17 @@ private void finishPrepare(UUID reqId, Map> res if (failure == null) { assert opCtx0 != null : ctx.localNodeId(); - Map filteredCfgs = new HashMap<>(); + Map globalCfgs = new HashMap<>(); for (List storedCfgs : res.values()) { if (storedCfgs == null) continue; for (StoredCacheData cacheData : storedCfgs) - filteredCfgs.put(cacheData.config().getName(), cacheData); + globalCfgs.put(CU.cacheId(cacheData.config().getName()), cacheData); } - opCtx0.configs(filteredCfgs.values()); + opCtx0.cfgs = globalCfgs; if (U.isLocalNodeCoordinator(ctx.discovery())) cacheStartProc.start(reqId, reqId); @@ -549,7 +588,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (opCtx0 == null) return new GridFinishedFuture<>(); - if (!reqId.equals(opCtx0.requestId())) + if (!reqId.equals(opCtx0.reqId)) return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); if (!U.isLocalNodeCoordinator(ctx.discovery())) @@ -560,16 +599,18 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (state.state() != ClusterState.ACTIVE || state.transition()) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); - if (opCtx0.interrupted()) - return new GridFinishedFuture<>(opCtx0.error()); + Throwable err = opCtx0.err.get(); - if (!allNodesInBaselineAndAlive(opCtx0.nodes())) + if (err != null) + return new GridFinishedFuture<>(err); + + if (!allNodesInBaselineAndAlive(opCtx0.nodes)) return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); GridFutureAdapter retFut = new GridFutureAdapter<>(); try { - Collection ccfgs = opCtx0.configs(); + Collection ccfgs = opCtx0.cfgs.values(); // Ensure that shared cache groups has no conflicts before start caches. for (StoredCacheData cfg : ccfgs) { @@ -579,15 +620,15 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (log.isInfoEnabled()) { log.info("Starting restored caches " + - "[snapshot=" + opCtx0.snapshotName() + - ", caches=" + F.viewReadOnly(opCtx0.configs(), c -> c.config().getName()) + ']'); + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + + ", caches=" + F.viewReadOnly(ccfgs, c -> c.config().getName()) + ']'); } - ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes()).listen( + ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes).listen( f -> { if (f.error() != null) { - log.error("Unable to start restored caches [groups=" + opCtx0.groups() + - ", snapshot=" + opCtx0.snapshotName() + ']', f.error()); + log.error("Unable to start restored caches [requestID=" + opCtx0.reqId + + ", snapshot=" + opCtx0.snpName + ']', f.error()); retFut.onDone(f.error()); } @@ -597,7 +638,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { ); } catch (Exception e) { log.error("Unable to restore cache group(s) from snapshot " + - "[groups=" + opCtx0.groups() + ", snapshot=" + opCtx0.snapshotName() + ']', e); + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', e); return new GridFinishedFuture<>(e); } @@ -613,7 +654,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { private void finishCacheStart(UUID reqId, Map res, Map errs) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 == null || !reqId.equals(opCtx0.requestId())) + if (opCtx0 == null || !reqId.equals(opCtx0.reqId)) return; Exception failure = checkFailure(errs, opCtx0, res.keySet()); @@ -639,8 +680,8 @@ private void finishCacheStart(UUID reqId, Map res, Map errs, SnapshotRestoreContext opCtx, Set respNodes) { Exception err = F.first(errs.values()); - if (err == null && opCtx != null && !respNodes.containsAll(opCtx.nodes())) { - Set leftNodes = new HashSet<>(opCtx.nodes()); + if (err == null && opCtx != null && !respNodes.containsAll(opCtx.nodes)) { + Set leftNodes = new HashSet<>(opCtx.nodes); leftNodes.removeAll(respNodes); @@ -657,15 +698,12 @@ private Exception checkFailure(Map errs, SnapshotRestoreContext private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 == null || !req.requestId().equals(opCtx0.requestId())) + if (opCtx0 == null || !req.requestId().equals(opCtx0.reqId)) return new GridFinishedFuture<>(); - if (!opCtx0.nodes().contains(ctx.localNodeId())) + if (!opCtx0.nodes.contains(ctx.localNodeId())) return new GridFinishedFuture<>(); - if (log.isInfoEnabled()) - log.info("Performing rollback routine for restored cache group(s) [groups=" + opCtx0.groups() + ']'); - rollback(opCtx0); return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse(req.error())); @@ -679,7 +717,7 @@ private IgniteInternalFuture rollback(SnapshotR private void finishRollback(UUID reqId, Map res, Map errs) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 == null || !reqId.equals(opCtx0.requestId())) + if (opCtx0 == null || !reqId.equals(opCtx0.reqId)) return; SnapshotRestoreRollbackResponse resp = F.first(F.viewReadOnly(res.values(), v -> v, Objects::nonNull)); @@ -715,110 +753,29 @@ private static class SnapshotRestoreContext { /** Baseline node IDs that must be alive to complete the operation. */ private final Set nodes; - /** List of cache group names to restore from the snapshot. */ - private final Set grps; - - /** Set of processed cache IDs. */ - private final Set cacheIds = new GridConcurrentHashSet<>(); + /** List of restored cache group directories. */ + private final Collection dirs; /** The exception that led to the interruption of the process. */ - private final AtomicReference errRef = new AtomicReference<>(); + private final AtomicReference err = new AtomicReference<>(); - /** Collection of cache configurations */ - private volatile List ccfgs; + /** Cache ID to configuration mapping. */ + private volatile Map cfgs; /** * @param reqId Request ID. * @param snpName Snapshot name. - * @param grps List of cache group names to restore from the snapshot. * @param nodes Baseline node IDs that must be alive to complete the operation. + * @param dirs List of cache group names to restore from the snapshot. + * @param cfgs Cache ID to configuration mapping. */ - protected SnapshotRestoreContext(UUID reqId, String snpName, Collection grps, Collection nodes) { + protected SnapshotRestoreContext(UUID reqId, String snpName, Collection nodes, Collection dirs, + Map cfgs) { this.reqId = reqId; - this.nodes = new HashSet<>(nodes); this.snpName = snpName; - - Set grps0 = new HashSet<>(); - - for (String grpName : grps) { - grps0.add(grpName); - cacheIds.add(CU.cacheId(grpName)); - } - - this.grps = grps0; - } - - /** @return Request ID. */ - protected UUID requestId() { - return reqId; - } - - /** @return Baseline node IDs that must be alive to complete the operation. */ - protected Set nodes() { - return Collections.unmodifiableSet(nodes); - } - - /** @return Snapshot name. */ - protected String snapshotName() { - return snpName; - } - - /** - * @return List of cache group names to restore from the snapshot. - */ - protected Set groups() { - return grps; - } - - /** - * @param name Cache name. - * @return {@code True} if the cache with the specified name is currently being restored. - */ - protected boolean containsCache(String name) { - return cacheIds.contains(CU.cacheId(name)); - } - - /** - * @param ccfgs Collection of cache configurations. - */ - protected void configs(Collection ccfgs) { - List ccfgs0 = new ArrayList<>(ccfgs.size()); - - for (StoredCacheData cacheData : ccfgs) { - ccfgs0.add(cacheData); - - if (cacheData.config().getGroupName() != null) - cacheIds.add(CU.cacheId(cacheData.config().getName())); - } - - this.ccfgs = ccfgs0; - } - - /** @return Collection of cache configurations */ - protected Collection configs() { - return ccfgs; - } - - /** - * @param err Error. - * @return {@code True} if operation has been interrupted by this call. - */ - protected boolean interrupt(Exception err) { - return errRef.compareAndSet(null, err); - } - - /** - * @return Interrupted flag. - */ - protected boolean interrupted() { - return error() != null; - } - - /** - * @return Error if operation was interrupted, otherwise {@code null}. - */ - protected @Nullable Throwable error() { - return errRef.get(); + this.nodes = new HashSet<>(nodes); + this.dirs = dirs; + this.cfgs = cfgs; } } } 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 8c65e0fe07b983..6c060b709944c7 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 @@ -431,7 +431,7 @@ private IgniteCache createCacheWithBinaryType( @Test public void testParallelCacheStartWithTheSameNameOnPrepare() throws Exception { checkCacheStartWithTheSameName(RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, IgniteCheckedException.class, - "Cache start failed. A cache named \"cache1\" is currently being restored from a snapshot."); + "Cache start failed. A cache or group with the same name is currently being restored from a snapshot"); } /** @@ -670,7 +670,7 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh assertNull("nodeIdx=" + nodeIdx + ", cache=" + cacheName, desc); GridTestUtils.waitForCondition( - () -> !grid.context().cache().context().snapshotMgr().isCacheRestoring(null), + () -> !grid.context().cache().context().snapshotMgr().isSnapshotRestoring(), TIMEOUT); File dir = resolveCacheDir(grid, cacheName); From c4f86266833b8d10b6cecc66877ff93c0ff48b97 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 16 Mar 2021 13:16:43 +0300 Subject: [PATCH 39/98] IGNITE-13805 Code cleanup. --- .../snapshot/SnapshotRestoreProcess.java | 194 +++++++++--------- .../snapshot/AbstractSnapshotSelfTest.java | 1 - 2 files changed, 102 insertions(+), 93 deletions(-) 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 11193d24adba2f..dd4359b6fa477e 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 @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.IdentityHashMap; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -34,6 +33,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -351,140 +351,150 @@ private IgniteInternalFuture> prepare(SnapshotRestore return new GridFinishedFuture<>(); try { - if (isSnapshotRestoring()) - throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - - DiscoveryDataClusterState state = ctx.state().clusterState(); - - if (state.state() != ClusterState.ACTIVE || state.transition()) - throw new IgniteException(OP_REJECT_MSG + "The cluster should be active."); - - if (!allNodesInBaselineAndAlive(req.nodes())) - throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + opCtx = prepareContext(req); + } catch (Exception e) { + return new GridFinishedFuture<>(e); + } - for (String grpName : req.groups()) - ensureCacheAbsent(grpName); + SnapshotRestoreContext opCtx0 = opCtx; - GridCacheSharedContext cctx = ctx.cache().context(); + if (opCtx0.dirs.isEmpty()) + return new GridFinishedFuture<>(); - SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); + if (log.isInfoEnabled()) { + log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + } - if (meta != null && meta.consistentId().equals(cctx.localNode().consistentId().toString()) - && meta.pageSize() != cctx.database().pageSize()) { - throw new IgniteCheckedException("Incompatible memory page size " + - "[snapshotPageSize=" + meta.pageSize() + - ", local=" + cctx.database().pageSize() + - ", snapshot=" + req.snapshotName() + - ", nodeId=" + cctx.localNodeId() + ']'); - } + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - String folderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - Map dirs = new IdentityHashMap<>(); - Map cfgMap = new HashMap<>(); + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + try { + restore(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); - // Collect cache configuration(s) and verify cache groups page size. - for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), folderName)) { - String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); + Throwable err = opCtx0.err.get(); - if (!req.groups().contains(grpName)) - continue; + if (err == null) { + retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); - ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(snpCacheDir, cfgMap); + return; + } - File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), - Paths.get(databaseRelativePath(folderName), snpCacheDir.getName()).toString(), false); + log.error("Snapshot restore process has been interrupted " + + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); - if (!cacheDir.exists()) - cacheDir.mkdir(); - else if (cacheDir.list().length > 0) { - throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + - "[group=" + grpName + ", dir=" + cacheDir + ']'); - } + rollback(opCtx0); - dirs.put(snpCacheDir, cacheDir); + retFut.onDone(err); } + catch (Throwable t) { + retFut.onDone(t); + } + }); - Map idsCfgMap = cfgMap.isEmpty() ? Collections.emptyMap() : - cfgMap.values().stream().collect(Collectors.toMap(v -> CU.cacheId(v.config().getName()), v -> v)); - - opCtx = new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), - new ArrayList<>(dirs.values()), idsCfgMap); + return retFut; + } - SnapshotRestoreContext opCtx0 = opCtx; + /** + * @param req Request to prepare cache group restore from the snapshot. + * @return Snapshot restore operation context. + * @throws IgniteCheckedException If failed. + */ + private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) throws IgniteCheckedException { + if (isSnapshotRestoring()) + throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - if (opCtx0.dirs.isEmpty()) - return new GridFinishedFuture<>(); + DiscoveryDataClusterState state = ctx.state().clusterState(); - if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + - ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); - } + if (state.state() != ClusterState.ACTIVE || state.transition()) + throw new IgniteException(OP_REJECT_MSG + "The cluster should be active."); - File binDir = binaryWorkDir(cctx.snapshotMgr().snapshotLocalDir(req.snapshotName()).getAbsolutePath(), folderName); + if (!allNodesInBaselineAndAlive(req.nodes())) + throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); - GridFutureAdapter> retFut = new GridFutureAdapter<>(); + for (String grpName : req.groups()) + ensureCacheAbsent(grpName); - cctx.snapshotMgr().snapshotExecutorService().execute(() -> { - try { - if (opCtx0.err.get() != null) - return; + GridCacheSharedContext cctx = ctx.cache().context(); - if (ctx.localNodeId().equals(req.updateMetaNodeId())) { - // Check binary metadata compatibility. - ctx.cacheObjects().checkMetadata(binDir); + SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); - // Cluster-wide update binary metadata. - ctx.cacheObjects().updateMetadata(binDir, () -> opCtx0.err.get() != null); - } + if (meta != null && meta.consistentId().equals(cctx.localNode().consistentId().toString()) + && meta.pageSize() != cctx.database().pageSize()) { + throw new IgniteCheckedException("Incompatible memory page size " + + "[snapshotPageSize=" + meta.pageSize() + + ", local=" + cctx.database().pageSize() + + ", snapshot=" + req.snapshotName() + + ", nodeId=" + cctx.localNodeId() + ']'); + } - copyPartitions(dirs, opCtx0); + String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); + List cacheDirs = new ArrayList<>(); + Map cfgsByName = new HashMap<>(); - Throwable err = opCtx0.err.get(); + // Collect cache configuration(s) and verify cache groups page size. + for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), pdsFolderName)) { + String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); - if (err == null) { - retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); + if (!req.groups().contains(grpName)) + continue; - return; - } + ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(snpCacheDir, cfgsByName); - log.error("Snapshot restore process has been interrupted " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); + File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), + Paths.get(databaseRelativePath(pdsFolderName), snpCacheDir.getName()).toString(), false); - rollback(opCtx0); + if (!cacheDir.exists()) + cacheDir.mkdir(); + else if (cacheDir.list().length > 0) { + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } - retFut.onDone(err); + cacheDirs.add(cacheDir); + } - } - catch (Throwable t) { - retFut.onDone(t); - } - }); + Map cfgsById = cfgsByName.isEmpty() ? Collections.emptyMap() : + cfgsByName.values().stream().collect(Collectors.toMap(v -> CU.cacheId(v.config().getName()), v -> v)); - return retFut; - } catch (Exception e) { - return new GridFinishedFuture<>(e); - } + return new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), cacheDirs, cfgsById); } /** - * Copy partition filess from the local snapshot directory. + * Copy partition files and update binary metadata. * - * @param dirs Cache directories. * @param opCtx Snapshot restore operation context. + * @param updateMeta Update binary metadata flag. * @throws IgniteCheckedException If failed. */ - protected void copyPartitions(Map dirs, - SnapshotRestoreContext opCtx) throws IgniteCheckedException { - for (Map.Entry cacheDirs : dirs.entrySet()) { - File snpCacheDir = cacheDirs.getKey(); - File cacheDir = cacheDirs.getValue(); + protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws IgniteCheckedException { + BooleanSupplier stopChecker = () -> opCtx.err.get() != null; + String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); + + if (updateMeta) { + File binDir = binaryWorkDir( + ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snpName).getAbsolutePath(), pdsFolderName); + + if (stopChecker.getAsBoolean()) + return; + + // Check binary metadata compatibility. + ctx.cacheObjects().checkMetadata(binDir); + + // Cluster-wide update binary metadata. + ctx.cacheObjects().updateMetadata(binDir, stopChecker); + } + + for (File cacheDir : opCtx.dirs) { + File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snpName), + Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); try { if (log.isInfoEnabled()) log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); for (File snpFile : snpCacheDir.listFiles()) { - if (opCtx.err.get() != null) + if (stopChecker.getAsBoolean()) return; File target = new File(cacheDir, snpFile.getName()); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 980a05802da891..d6bb70975d4c3b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -135,7 +135,6 @@ public abstract class AbstractSnapshotSelfTest extends GridCommonAbstractTest { .setPersistenceEnabled(persistence)) .setCheckpointFrequency(3000) .setPageSize(DFLT_PAGE_SIZE)) - .setCacheConfiguration(dfltCacheCfg) .setClusterStateOnStart(INACTIVE) .setIncludeEventTypes(EVTS_CLUSTER_SNAPSHOT) .setDiscoverySpi(discoSpi); From 2eefae883d4d754ece19de86366aab1a44656541 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 16 Mar 2021 17:28:59 +0300 Subject: [PATCH 40/98] IGNITE-13805 Catch ignitecheckedexception only in prepare. --- .../snapshot/SnapshotRestoreProcess.java | 72 ++++++++++--------- 1 file changed, 40 insertions(+), 32 deletions(-) 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 dd4359b6fa477e..bc6e6665c200df 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 @@ -121,16 +121,16 @@ public SnapshotRestoreProcess(GridKernalContext ctx) { */ public IgniteFuture start(String snpName, Collection cacheGrpNames) { if (ctx.clientNode()) { - return new IgniteFinishedFutureImpl<>(new UnsupportedOperationException("Client and daemon nodes can not " + - "perform this operation.")); + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "Client and daemon nodes can not perform this operation.")); } synchronized (this) { GridFutureAdapter fut0 = fut; if (opCtx != null || (fut0 != null && !fut0.isDone())) { - return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + - "The previous snapshot restore operation was not completed.")); + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); } fut = new GridFutureAdapter<>(); @@ -142,19 +142,21 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); if (!clusterState.hasBaselineTopology()) { - return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + - "The baseline topology is not configured for cluster.")); + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "The baseline topology is not configured for cluster.")); } IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); if (snpMgr.isSnapshotCreating()) { - return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + - "A cluster snapshot operation is in progress.")); + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); } - if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) - throw new IgniteException("Not all nodes in the cluster support a snapshot restore operation."); + if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) { + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation.")); + } snpMgr.collectSnapshotMetadata(snpName).listen( f -> { @@ -332,12 +334,13 @@ public void stop(Exception reason) { * Ensures that a cache with the specified name does not exist locally. * * @param name Cache name. + * @throws IgniteCheckedException If cache is present. */ - private void ensureCacheAbsent(String name) { + private void ensureCacheAbsent(String name) throws IgniteCheckedException { int id = CU.cacheId(name); if (ctx.cache().cacheGroupDescriptors().containsKey(id) || ctx.cache().cacheDescriptor(id) != null) { - throw new IllegalStateException("Cache \"" + name + + throw new IgniteCheckedException("Cache \"" + name + "\" should be destroyed manually before perform restore operation."); } } @@ -351,8 +354,19 @@ private IgniteInternalFuture> prepare(SnapshotRestore return new GridFinishedFuture<>(); try { + DiscoveryDataClusterState state = ctx.state().clusterState(); + + if (state.state() != ClusterState.ACTIVE || state.transition()) + throw new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active."); + + if (!allNodesInBaselineAndAlive(req.nodes())) + throw new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + + for (String grpName : req.groups()) + ensureCacheAbsent(grpName); + opCtx = prepareContext(req); - } catch (Exception e) { + } catch (IgniteCheckedException e) { return new GridFinishedFuture<>(e); } @@ -401,19 +415,10 @@ private IgniteInternalFuture> prepare(SnapshotRestore * @throws IgniteCheckedException If failed. */ private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) throws IgniteCheckedException { - if (isSnapshotRestoring()) - throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - - DiscoveryDataClusterState state = ctx.state().clusterState(); - - if (state.state() != ClusterState.ACTIVE || state.transition()) - throw new IgniteException(OP_REJECT_MSG + "The cluster should be active."); - - if (!allNodesInBaselineAndAlive(req.nodes())) - throw new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster."); - - for (String grpName : req.groups()) - ensureCacheAbsent(grpName); + if (isSnapshotRestoring()) { + throw new IgniteCheckedException(OP_REJECT_MSG + + "The previous snapshot restore operation was not completed."); + } GridCacheSharedContext cctx = ctx.cache().context(); @@ -598,8 +603,10 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (opCtx0 == null) return new GridFinishedFuture<>(); - if (!reqId.equals(opCtx0.reqId)) - return new GridFinishedFuture<>(new IgniteException("Unknown snapshot restore operation was rejected.")); + if (!reqId.equals(opCtx0.reqId)) { + return new GridFinishedFuture<>( + new IgniteCheckedException("Unknown snapshot restore operation was rejected.")); + } if (!U.isLocalNodeCoordinator(ctx.discovery())) return new GridFinishedFuture<>(); @@ -607,7 +614,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { DiscoveryDataClusterState state = ctx.state().clusterState(); if (state.state() != ClusterState.ACTIVE || state.transition()) - return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); + return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active.")); Throwable err = opCtx0.err.get(); @@ -615,7 +622,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { return new GridFinishedFuture<>(err); if (!allNodesInBaselineAndAlive(opCtx0.nodes)) - return new GridFinishedFuture<>(new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); + return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); GridFutureAdapter retFut = new GridFutureAdapter<>(); @@ -646,7 +653,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { retFut.onDone(true); } ); - } catch (Exception e) { + } catch (IgniteCheckedException e) { log.error("Unable to restore cache group(s) from snapshot " + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', e); @@ -695,7 +702,8 @@ private Exception checkFailure(Map errs, SnapshotRestoreContext leftNodes.removeAll(respNodes); - err = new IgniteException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); + err = new IgniteCheckedException(OP_REJECT_MSG + + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); } return err; From 64c92ac7934934bd9a745ba644387dbbced7ea1a Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 17 Mar 2021 10:51:07 +0300 Subject: [PATCH 41/98] IGNITE-13805 Review notes. --- .../CacheObjectBinaryProcessorImpl.java | 31 +- .../snapshot/IgniteSnapshotManager.java | 2 +- .../snapshot/SnapshotRestoreProcess.java | 268 ++++++++---------- .../IgniteCacheObjectProcessor.java | 8 - 4 files changed, 133 insertions(+), 176 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 69d6acf6f638be..ce1a2cb1c90ab7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -995,22 +995,27 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { } /** {@inheritDoc} */ - @Override public void checkMetadata(File metadataDir) throws IgniteCheckedException { - for (BinaryMetadata newMeta : readMetadata(metadataDir)) { - BinaryMetadata oldMeta = binaryMetadata(newMeta.typeId()); + @Override public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException { + Collection metadata = readMetadata(metadataDir); - if (oldMeta != null) - BinaryUtils.mergeMetadata(oldMeta, newMeta, null); - } - } + try { + // Check the compatibility of the binary metadata files stored in the specified directory. + for (BinaryMetadata newMeta : metadata) { + BinaryMetadata oldMeta = binaryMetadata(newMeta.typeId()); - /** {@inheritDoc} */ - @Override public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException { - for (BinaryMetadata newMeta : readMetadata(metadataDir)) { - if (stopChecker.getAsBoolean()) - return; + if (oldMeta != null) + BinaryUtils.mergeMetadata(oldMeta, newMeta, null); + } - addMeta(newMeta.typeId(), newMeta.wrap(binaryContext()), false); + // Update cluster metadata. + for (BinaryMetadata newMeta : metadata) { + if (stopChecker.getAsBoolean()) + return; + + addMeta(newMeta.typeId(), newMeta.wrap(binaryContext()), false); + } + } catch (BinaryObjectException e) { + throw new IgniteCheckedException(e); } } 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 fb56874e175827..c3c23c413b7ad9 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 @@ -1372,7 +1372,7 @@ private void recordSnapshotEvent(String snpName, String msg, int type) { /** * @return The executor used to run snapshot tasks. */ - Executor snapshotExecutorService() { + ExecutorService snapshotExecutorService() { assert snpRunner != null; return snpRunner; 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 bc6e6665c200df..35005c2952512d 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 @@ -22,6 +22,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -29,14 +30,15 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteLogger; import org.apache.ignite.cluster.ClusterNode; import org.apache.ignite.cluster.ClusterState; @@ -61,6 +63,7 @@ import static org.apache.ignite.internal.IgniteFeatures.SNAPSHOT_RESTORE_CACHE_GROUP; import static org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.binaryWorkDir; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.databaseRelativePath; 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_ROLLBACK; @@ -334,13 +337,12 @@ public void stop(Exception reason) { * Ensures that a cache with the specified name does not exist locally. * * @param name Cache name. - * @throws IgniteCheckedException If cache is present. */ - private void ensureCacheAbsent(String name) throws IgniteCheckedException { + private void ensureCacheAbsent(String name) { int id = CU.cacheId(name); if (ctx.cache().cacheGroupDescriptors().containsKey(id) || ctx.cache().cacheDescriptor(id) != null) { - throw new IgniteCheckedException("Cache \"" + name + + throw new IgniteIllegalStateException("Cache \"" + name + "\" should be destroyed manually before perform restore operation."); } } @@ -359,54 +361,64 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (state.state() != ClusterState.ACTIVE || state.transition()) throw new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active."); - if (!allNodesInBaselineAndAlive(req.nodes())) - throw new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + for (UUID nodeId : req.nodes()) { + ClusterNode node = ctx.discovery().node(nodeId); + + if (node == null || !CU.baselineNode(node, state) || !ctx.discovery().alive(node)) + throw new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster."); + } for (String grpName : req.groups()) ensureCacheAbsent(grpName); opCtx = prepareContext(req); - } catch (IgniteCheckedException e) { - return new GridFinishedFuture<>(e); - } - SnapshotRestoreContext opCtx0 = opCtx; + SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0.dirs.isEmpty()) - return new GridFinishedFuture<>(); + if (opCtx0.dirs.isEmpty()) + return new GridFinishedFuture<>(); - if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + - ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); - } + // Ensure that shared cache groups has no conflicts. + for (StoredCacheData cfg : opCtx0.cfgs.values()) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - GridFutureAdapter> retFut = new GridFutureAdapter<>(); + if (log.isInfoEnabled()) { + log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + } - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - try { - restore(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - Throwable err = opCtx0.err.get(); + ctx.getSystemExecutorService().execute(() -> { + try { + restore(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); - if (err == null) { - retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); + Throwable err = opCtx0.err.get(); - return; - } + if (err == null) { + retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); - log.error("Snapshot restore process has been interrupted " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); + return; + } - rollback(opCtx0); + log.error("Snapshot restore process has been interrupted " + + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); - retFut.onDone(err); - } - catch (Throwable t) { - retFut.onDone(t); - } - }); + rollback(opCtx0); + + retFut.onDone(err); + } + catch (IgniteCheckedException e) { + retFut.onDone(e); + } + }); - return retFut; + return retFut; + } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { + return new GridFinishedFuture<>(e); + } } /** @@ -424,8 +436,10 @@ private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) SnapshotMetadata meta = F.first(cctx.snapshotMgr().readSnapshotMetadatas(req.snapshotName())); - if (meta != null && meta.consistentId().equals(cctx.localNode().consistentId().toString()) - && meta.pageSize() != cctx.database().pageSize()) { + if (meta == null || !meta.consistentId().equals(cctx.localNode().consistentId().toString())) + return new SnapshotRestoreContext(req, Collections.emptyList(), Collections.emptyMap()); + + if (meta.pageSize() != cctx.database().pageSize()) { throw new IgniteCheckedException("Incompatible memory page size " + "[snapshotPageSize=" + meta.pageSize() + ", local=" + cctx.database().pageSize() + @@ -433,21 +447,18 @@ private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) ", nodeId=" + cctx.localNodeId() + ']'); } - String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); List cacheDirs = new ArrayList<>(); Map cfgsByName = new HashMap<>(); + FilePageStoreManager pageStore = (FilePageStoreManager)cctx.pageStore(); // Collect cache configuration(s) and verify cache groups page size. - for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), pdsFolderName)) { + for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), meta.folderName())) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); if (!req.groups().contains(grpName)) continue; - ((FilePageStoreManager)cctx.pageStore()).readCacheConfigurations(snpCacheDir, cfgsByName); - - File cacheDir = U.resolveWorkDirectory(ctx.config().getWorkDirectory(), - Paths.get(databaseRelativePath(pdsFolderName), snpCacheDir.getName()).toString(), false); + File cacheDir = pageStore.cacheWorkDir(snpCacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX), grpName); if (!cacheDir.exists()) cacheDir.mkdir(); @@ -457,12 +468,14 @@ else if (cacheDir.list().length > 0) { } cacheDirs.add(cacheDir); + + pageStore.readCacheConfigurations(snpCacheDir, cfgsByName); } Map cfgsById = cfgsByName.isEmpty() ? Collections.emptyMap() : cfgsByName.values().stream().collect(Collectors.toMap(v -> CU.cacheId(v.config().getName()), v -> v)); - return new SnapshotRestoreContext(req.requestId(), req.snapshotName(), req.nodes(), cacheDirs, cfgsById); + return new SnapshotRestoreContext(req, cacheDirs, cfgsById); } /** @@ -473,7 +486,7 @@ else if (cacheDir.list().length > 0) { * @throws IgniteCheckedException If failed. */ protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws IgniteCheckedException { - BooleanSupplier stopChecker = () -> opCtx.err.get() != null; + BooleanSupplier stopChecker = () -> opCtx.err.get() != null || Thread.currentThread().isInterrupted(); String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); if (updateMeta) { @@ -483,10 +496,6 @@ protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws if (stopChecker.getAsBoolean()) return; - // Check binary metadata compatibility. - ctx.cacheObjects().checkMetadata(binDir); - - // Cluster-wide update binary metadata. ctx.cacheObjects().updateMetadata(binDir, stopChecker); } @@ -494,13 +503,15 @@ protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snpName), Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); - try { - if (log.isInfoEnabled()) - log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); + if (log.isInfoEnabled()) + log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); - for (File snpFile : snpCacheDir.listFiles()) { + U.doInParallel( + ctx.cache().context().snapshotMgr().snapshotExecutorService(), + Arrays.asList(snpCacheDir.listFiles()), + snpFile -> { if (stopChecker.getAsBoolean()) - return; + return null; File target = new File(cacheDir, snpFile.getName()); @@ -511,13 +522,17 @@ protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws ", target=" + target + "]"); } - Files.copy(snpFile.toPath(), target.toPath()); + try { + Files.copy(snpFile.toPath(), target.toPath()); + } + catch (IOException e) { + throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snpName + + ", grp=" + FilePageStoreManager.cacheGroupName(cacheDir) + ']', e); + } + + return null; } - } - catch (IOException e) { - throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snpName + - ", grp=" + FilePageStoreManager.cacheGroupName(cacheDir) + ']', e); - } + ); } } @@ -533,19 +548,14 @@ private void rollback(@Nullable SnapshotRestoreContext opCtx) { if (log.isInfoEnabled()) log.info("Performing local rollback routine for restored cache groups [requestID=" + opCtx.reqId + ']'); - try { - for (File cacheDir : opCtx.dirs) { - if (!cacheDir.exists()) - continue; + for (File cacheDir : opCtx.dirs) { + if (!cacheDir.exists()) + continue; - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); - U.delete(cacheDir); - } - } - catch (Exception e) { - log.error("Failed to perform rollback [requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', e); + U.delete(cacheDir); } } @@ -585,12 +595,8 @@ private void finishPrepare(UUID reqId, Map> res if (opCtx0 == null) finishProcess(failure); - else // Remove files asynchronously. - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - rollback(opCtx0); - - finishProcess(failure); - }); + else if (U.isLocalNodeCoordinator(ctx.discovery())) + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); } /** @@ -598,15 +604,12 @@ private void finishPrepare(UUID reqId, Map> res * @return Result future. */ private IgniteInternalFuture cacheStart(UUID reqId) { - SnapshotRestoreContext opCtx0 = opCtx; - - if (opCtx0 == null) + if (ctx.clientNode()) return new GridFinishedFuture<>(); - if (!reqId.equals(opCtx0.reqId)) { - return new GridFinishedFuture<>( - new IgniteCheckedException("Unknown snapshot restore operation was rejected.")); - } + SnapshotRestoreContext opCtx0 = opCtx; + + assert opCtx0 != null : ctx.localNodeId(); if (!U.isLocalNodeCoordinator(ctx.discovery())) return new GridFinishedFuture<>(); @@ -621,46 +624,15 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (err != null) return new GridFinishedFuture<>(err); - if (!allNodesInBaselineAndAlive(opCtx0.nodes)) - return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster.")); - - GridFutureAdapter retFut = new GridFutureAdapter<>(); - - try { - Collection ccfgs = opCtx0.cfgs.values(); - - // Ensure that shared cache groups has no conflicts before start caches. - for (StoredCacheData cfg : ccfgs) { - if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); - } - - if (log.isInfoEnabled()) { - log.info("Starting restored caches " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + - ", caches=" + F.viewReadOnly(ccfgs, c -> c.config().getName()) + ']'); - } - - ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes).listen( - f -> { - if (f.error() != null) { - log.error("Unable to start restored caches [requestID=" + opCtx0.reqId + - ", snapshot=" + opCtx0.snpName + ']', f.error()); - - retFut.onDone(f.error()); - } - else - retFut.onDone(true); - } - ); - } catch (IgniteCheckedException e) { - log.error("Unable to restore cache group(s) from snapshot " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', e); + Collection ccfgs = opCtx0.cfgs.values(); - return new GridFinishedFuture<>(e); + if (log.isInfoEnabled()) { + log.info("Starting restored caches " + + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + + ", caches=" + F.viewReadOnly(ccfgs, c -> c.config().getName()) + ']'); } - return retFut; + return ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes); } /** @@ -669,11 +641,11 @@ private IgniteInternalFuture cacheStart(UUID reqId) { * @param errs Errors. */ private void finishCacheStart(UUID reqId, Map res, Map errs) { - SnapshotRestoreContext opCtx0 = opCtx; - - if (opCtx0 == null || !reqId.equals(opCtx0.reqId)) + if (ctx.clientNode()) return; + SnapshotRestoreContext opCtx0 = opCtx; + Exception failure = checkFailure(errs, opCtx0, res.keySet()); if (failure == null) { @@ -714,15 +686,10 @@ private Exception checkFailure(Map errs, SnapshotRestoreContext * @return Result future. */ private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { - SnapshotRestoreContext opCtx0 = opCtx; - - if (opCtx0 == null || !req.requestId().equals(opCtx0.reqId)) - return new GridFinishedFuture<>(); - - if (!opCtx0.nodes.contains(ctx.localNodeId())) + if (ctx.clientNode()) return new GridFinishedFuture<>(); - rollback(opCtx0); + rollback(opCtx); return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse(req.error())); } @@ -733,29 +700,23 @@ private IgniteInternalFuture rollback(SnapshotR * @param errs Errors. */ private void finishRollback(UUID reqId, Map res, Map errs) { - SnapshotRestoreContext opCtx0 = opCtx; - - if (opCtx0 == null || !reqId.equals(opCtx0.reqId)) + if (ctx.clientNode()) return; - SnapshotRestoreRollbackResponse resp = F.first(F.viewReadOnly(res.values(), v -> v, Objects::nonNull)); + SnapshotRestoreContext opCtx0 = opCtx; - finishProcess(resp.error()); - } + if (!res.keySet().containsAll(opCtx0.nodes)) { + Set leftNodes = new HashSet<>(opCtx0.nodes); - /** - * @param nodeIds Set of required baseline node IDs. - * @return {@code True} if all of the specified nodes present in baseline and alive. - */ - private boolean allNodesInBaselineAndAlive(Set nodeIds) { - for (UUID nodeId : nodeIds) { - ClusterNode node = ctx.discovery().node(nodeId); + leftNodes.removeAll(res.keySet()); - if (node == null || !CU.baselineNode(node, ctx.state().clusterState()) || !ctx.discovery().alive(node)) - return false; + log.warning("Some of the nodes left the cluster and were unable to complete the rollback" + + " operation [requestID=" + reqId + ", snapshot=" + opCtx0.snpName + ", node(s)=" + leftNodes + ']'); } - return true; + SnapshotRestoreRollbackResponse resp = F.first(res.values()); + + finishProcess(resp.error()); } /** @@ -781,17 +742,16 @@ private static class SnapshotRestoreContext { private volatile Map cfgs; /** - * @param reqId Request ID. - * @param snpName Snapshot name. - * @param nodes Baseline node IDs that must be alive to complete the operation. + * @param req Request to prepare cache group restore from the snapshot. * @param dirs List of cache group names to restore from the snapshot. * @param cfgs Cache ID to configuration mapping. */ - protected SnapshotRestoreContext(UUID reqId, String snpName, Collection nodes, Collection dirs, + protected SnapshotRestoreContext(SnapshotRestorePrepareRequest req, Collection dirs, Map cfgs) { - this.reqId = reqId; - this.snpName = snpName; - this.nodes = new HashSet<>(nodes); + reqId = req.requestId(); + snpName = req.snapshotName(); + nodes = new HashSet<>(req.nodes()); + this.dirs = dirs; this.cfgs = cfgs; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java index 37956bb42d5a0d..a4c9eb4fc4a1dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java @@ -307,14 +307,6 @@ public void updateMetadata(int typeId, String typeName, @Nullable String affKeyF */ public void saveMetadata(Collection types, File dir); - /** - * Check the compatibility of the binary metadata files stored in the specified directory. - * - * @param metadataDir Directory containing binary metadata files. - * @throws IgniteCheckedException If failed. - */ - public void checkMetadata(File metadataDir) throws IgniteCheckedException; - /** * Merge the binary metadata files stored in the specified directory. * From 154e0792839802e0921b0c70ad5f833166e17ec4 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 17 Mar 2021 19:09:45 +0300 Subject: [PATCH 42/98] IGNITE-13805 Additional errors logging. --- .../snapshot/SnapshotRestoreProcess.java | 19 +++++++++++-------- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 ++-- 2 files changed, 13 insertions(+), 10 deletions(-) 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 35005c2952512d..7958043c55b1a2 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 @@ -411,12 +411,18 @@ private IgniteInternalFuture> prepare(SnapshotRestore retFut.onDone(err); } catch (IgniteCheckedException e) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', e); + retFut.onDone(e); } }); return retFut; } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[requestID=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); + return new GridFinishedFuture<>(e); } } @@ -537,7 +543,7 @@ protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws } /** - * Rollback changes made by process in specified cache group. + * Rollback changes made by process. * * @param opCtx Snapshot restore operation context. */ @@ -545,8 +551,10 @@ private void rollback(@Nullable SnapshotRestoreContext opCtx) { if (opCtx == null || F.isEmpty(opCtx.dirs)) return; - if (log.isInfoEnabled()) - log.info("Performing local rollback routine for restored cache groups [requestID=" + opCtx.reqId + ']'); + if (log.isInfoEnabled()) { + log.info("Performing local rollback routine for restored cache groups " + + "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']'); + } for (File cacheDir : opCtx.dirs) { if (!cacheDir.exists()) @@ -614,11 +622,6 @@ private IgniteInternalFuture cacheStart(UUID reqId) { if (!U.isLocalNodeCoordinator(ctx.discovery())) return new GridFinishedFuture<>(); - DiscoveryDataClusterState state = ctx.state().clusterState(); - - if (state.state() != ClusterState.ACTIVE || state.transition()) - return new GridFinishedFuture<>(new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active.")); - Throwable err = opCtx0.err.get(); if (err != null) 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 6c060b709944c7..94359f0dc6d403 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 @@ -551,7 +551,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { @Test public void testClusterStateChangeActiveReadonlyOnPrepare() throws Exception { checkClusterStateChange(ClusterState.ACTIVE_READ_ONLY, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - IgniteException.class, "The cluster should be active."); + IgniteException.class, "Failed to perform start cache operation (cluster is in read-only mode)"); } /** @@ -568,7 +568,7 @@ public void testClusterStateChangeActiveReadonlyOnCacheStart() throws Exception @Test public void testClusterDeactivateOnPrepare() throws Exception { checkClusterStateChange(ClusterState.INACTIVE, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, - IgniteException.class, "The cluster should be active."); + IgniteException.class, "The cluster has been deactivated."); } /** From 4d1f056fe6488a75f9828e4c616d5719bf20af93 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 18 Mar 2021 19:50:16 +0300 Subject: [PATCH 43/98] IGNITE-13805 More complicated concurrency. --- .../snapshot/IgniteSnapshotManager.java | 2 +- .../snapshot/SnapshotRestoreProcess.java | 168 +++++++++--------- 2 files changed, 87 insertions(+), 83 deletions(-) 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 c3c23c413b7ad9..fb56874e175827 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 @@ -1372,7 +1372,7 @@ private void recordSnapshotEvent(String snpName, String msg, int type) { /** * @return The executor used to run snapshot tasks. */ - ExecutorService snapshotExecutorService() { + Executor snapshotExecutorService() { assert snpRunner != null; return snpRunner; 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 7958043c55b1a2..7479f9298041e2 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 @@ -22,7 +22,6 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -32,6 +31,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; @@ -45,6 +45,7 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; @@ -389,42 +390,103 @@ private IgniteInternalFuture> prepare(SnapshotRestore ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); } - GridFutureAdapter> retFut = new GridFutureAdapter<>(); + return restoreAsync(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); - ctx.getSystemExecutorService().execute(() -> { - try { - restore(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); + } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[requestID=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); + + return new GridFinishedFuture<>(e); + } + } + + /** + * Copy partition files and update binary metadata. + * + * @param opCtx Snapshot restore operation context. + * @param updateMeta Update binary metadata flag. + * @throws IgniteCheckedException If failed. + */ + private IgniteInternalFuture> restoreAsync( + SnapshotRestoreContext opCtx, + boolean updateMeta + ) throws IgniteCheckedException { + IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); + String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - Throwable err = opCtx0.err.get(); + BooleanSupplier stopChecker = () -> { + if (opCtx.err.get() != null) + return true; - if (err == null) { - retFut.onDone(new ArrayList<>(opCtx0.cfgs.values())); + if (Thread.currentThread().isInterrupted()) { + opCtx.err.compareAndSet(null, new IgniteInterruptedCheckedException("Thread has been interrupted.")); - return; - } + return true; + } + + return false; + }; + + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - log.error("Snapshot restore process has been interrupted " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', err); + List> futs = new ArrayList<>(); - rollback(opCtx0); + if (updateMeta) { + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(opCtx.snpName).getAbsolutePath(), pdsFolderName); - retFut.onDone(err); + futs.add(CompletableFuture.runAsync(() -> { + try { + ctx.cacheObjects().updateMetadata(binDir, stopChecker); } catch (IgniteCheckedException e) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']', e); - - retFut.onDone(e); + opCtx.err.compareAndSet(null, e); } - }); + }, snapshotMgr.snapshotExecutorService())); + } - return retFut; - } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[requestID=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); + for (File cacheDir : this.opCtx.dirs) { + File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(this.opCtx.snpName), + Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); - return new GridFinishedFuture<>(e); + assert snpCacheDir.exists() : "node=" + ctx.localNodeId() + ", dir=" + snpCacheDir; + + for (File snpFile : snpCacheDir.listFiles()) { + futs.add(CompletableFuture.runAsync(() -> { + if (stopChecker.getAsBoolean()) + return; + + File target = new File(cacheDir, snpFile.getName()); + + if (log.isDebugEnabled()) { + log.debug("Copying file from the snapshot " + + "[snapshot=" + this.opCtx.snpName + + ", src=" + snpFile + + ", target=" + target + "]"); + } + + try { + Files.copy(snpFile.toPath(), target.toPath()); + } + catch (IOException e) { + opCtx.err.compareAndSet(null, e); + } + }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); + } } + + CompletableFuture.allOf(futs.toArray(new CompletableFuture[0])).thenAccept(res -> { + Throwable err = opCtx.err.get(); + + if (err != null) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + + retFut.onDone(err); + } else + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + }); + + return retFut; } /** @@ -484,64 +546,6 @@ else if (cacheDir.list().length > 0) { return new SnapshotRestoreContext(req, cacheDirs, cfgsById); } - /** - * Copy partition files and update binary metadata. - * - * @param opCtx Snapshot restore operation context. - * @param updateMeta Update binary metadata flag. - * @throws IgniteCheckedException If failed. - */ - protected void restore(SnapshotRestoreContext opCtx, boolean updateMeta) throws IgniteCheckedException { - BooleanSupplier stopChecker = () -> opCtx.err.get() != null || Thread.currentThread().isInterrupted(); - String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - - if (updateMeta) { - File binDir = binaryWorkDir( - ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snpName).getAbsolutePath(), pdsFolderName); - - if (stopChecker.getAsBoolean()) - return; - - ctx.cacheObjects().updateMetadata(binDir, stopChecker); - } - - for (File cacheDir : opCtx.dirs) { - File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(opCtx.snpName), - Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); - - if (log.isInfoEnabled()) - log.info("Copying files of the cache group [from=" + snpCacheDir + ", to=" + cacheDir + ']'); - - U.doInParallel( - ctx.cache().context().snapshotMgr().snapshotExecutorService(), - Arrays.asList(snpCacheDir.listFiles()), - snpFile -> { - if (stopChecker.getAsBoolean()) - return null; - - File target = new File(cacheDir, snpFile.getName()); - - if (log.isDebugEnabled()) { - log.debug("Copying file from the snapshot " + - "[snapshot=" + opCtx.snpName + - ", src=" + snpFile + - ", target=" + target + "]"); - } - - try { - Files.copy(snpFile.toPath(), target.toPath()); - } - catch (IOException e) { - throw new IgniteCheckedException("Unable to copy file [snapshot=" + opCtx.snpName + - ", grp=" + FilePageStoreManager.cacheGroupName(cacheDir) + ']', e); - } - - return null; - } - ); - } - } - /** * Rollback changes made by process. * From 1afc06d3a78d87c0de748845fa62033aac751df3 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 18 Mar 2021 21:58:50 +0300 Subject: [PATCH 44/98] IGNITE-13805 checkFailure -> checkNodeLeft --- .../snapshot/SnapshotRestoreProcess.java | 69 ++++++++++--------- 1 file changed, 37 insertions(+), 32 deletions(-) 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 7479f9298041e2..7da78b7482d6fe 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 @@ -577,38 +577,45 @@ private void rollback(@Nullable SnapshotRestoreContext opCtx) { * @param errs Errors. */ private void finishPrepare(UUID reqId, Map> res, Map errs) { - SnapshotRestoreContext opCtx0 = opCtx; - if (ctx.clientNode()) return; - Exception failure = checkFailure(errs, opCtx0, res.keySet()); + SnapshotRestoreContext opCtx0 = opCtx; - if (failure == null) { - assert opCtx0 != null : ctx.localNodeId(); + Exception failure = F.first(errs.values()); - Map globalCfgs = new HashMap<>(); + assert opCtx0 != null || failure != null : ctx.localNodeId(); - for (List storedCfgs : res.values()) { - if (storedCfgs == null) - continue; + if (opCtx0 == null) { + finishProcess(failure); - for (StoredCacheData cacheData : storedCfgs) - globalCfgs.put(CU.cacheId(cacheData.config().getName()), cacheData); - } + return; + } - opCtx0.cfgs = globalCfgs; + if (failure == null) + failure = checNodeLeft(opCtx0.nodes, res.keySet()); - if (U.isLocalNodeCoordinator(ctx.discovery())) - cacheStartProc.start(reqId, reqId); + // Context has been created - should rollback changes cluster-wide. + if (failure != null && U.isLocalNodeCoordinator(ctx.discovery())) { + rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); return; } - if (opCtx0 == null) - finishProcess(failure); - else if (U.isLocalNodeCoordinator(ctx.discovery())) - rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); + Map globalCfgs = new HashMap<>(); + + for (List storedCfgs : res.values()) { + if (storedCfgs == null) + continue; + + for (StoredCacheData cacheData : storedCfgs) + globalCfgs.put(CU.cacheId(cacheData.config().getName()), cacheData); + } + + opCtx0.cfgs = globalCfgs; + + if (U.isLocalNodeCoordinator(ctx.discovery())) + cacheStartProc.start(reqId, reqId); } /** @@ -653,7 +660,10 @@ private void finishCacheStart(UUID reqId, Map res, Map res, Map errs, SnapshotRestoreContext opCtx, Set respNodes) { - Exception err = F.first(errs.values()); - - if (err == null && opCtx != null && !respNodes.containsAll(opCtx.nodes)) { - Set leftNodes = new HashSet<>(opCtx.nodes); + private Exception checNodeLeft(Set reqNodes, Set respNodes) { + if (!respNodes.containsAll(reqNodes)) { + Set leftNodes = new HashSet<>(reqNodes); leftNodes.removeAll(respNodes); - err = new IgniteCheckedException(OP_REJECT_MSG + + return new IgniteCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); } - return err; + return null; } /** From 310312bde5bcf556258e85e1d60dde2c866d20ca Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 18 Mar 2021 22:30:24 +0300 Subject: [PATCH 45/98] IGNITE-13805 Removed redundant rollback request/respnses. --- .../snapshot/SnapshotRestoreProcess.java | 28 ++++---- .../SnapshotRestoreRollbackRequest.java | 65 ------------------- .../SnapshotRestoreRollbackResponse.java | 45 ------------- 3 files changed, 16 insertions(+), 122 deletions(-) delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackRequest.java delete mode 100644 modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRollbackResponse.java 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 7da78b7482d6fe..61aa4ae0db1fab 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 @@ -46,6 +46,7 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; @@ -87,7 +88,7 @@ public class SnapshotRestoreProcess { private final DistributedProcess cacheStartProc; /** Cache group restore rollback phase. */ - private final DistributedProcess rollbackRestoreProc; + private final DistributedProcess rollbackRestoreProc; /** Logger. */ private final IgniteLogger log; @@ -317,7 +318,7 @@ public void onNodeLeft(UUID leftNodeId) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 != null && opCtx0.nodes.contains(leftNodeId)) { - opCtx0.err.compareAndSet(null, new IgniteException(OP_REJECT_MSG + + opCtx0.err.compareAndSet(null, new ClusterTopologyCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodeId + ']')); } } @@ -596,8 +597,11 @@ private void finishPrepare(UUID reqId, Map> res failure = checNodeLeft(opCtx0.nodes, res.keySet()); // Context has been created - should rollback changes cluster-wide. - if (failure != null && U.isLocalNodeCoordinator(ctx.discovery())) { - rollbackRestoreProc.start(reqId, new SnapshotRestoreRollbackRequest(reqId, failure)); + if (failure != null) { + opCtx0.err.compareAndSet(null, failure); + + if (U.isLocalNodeCoordinator(ctx.discovery())) + rollbackRestoreProc.start(reqId, reqId); return; } @@ -671,8 +675,10 @@ private void finishCacheStart(UUID reqId, Map res, Map reqNodes, Set respNodes) { } /** - * @param req Request to rollback cache group restore process. + * @param reqId Request ID. * @return Result future. */ - private IgniteInternalFuture rollback(SnapshotRestoreRollbackRequest req) { + private IgniteInternalFuture rollback(UUID reqId) { if (ctx.clientNode()) return new GridFinishedFuture<>(); rollback(opCtx); - return new GridFinishedFuture<>(new SnapshotRestoreRollbackResponse(req.error())); + return new GridFinishedFuture<>(true); } /** @@ -711,7 +717,7 @@ private IgniteInternalFuture rollback(SnapshotR * @param res Results. * @param errs Errors. */ - private void finishRollback(UUID reqId, Map res, Map errs) { + private void finishRollback(UUID reqId, Map res, Map errs) { if (ctx.clientNode()) return; @@ -726,9 +732,7 @@ private void finishRollback(UUID reqId, Map Date: Thu, 18 Mar 2021 22:39:41 +0300 Subject: [PATCH 46/98] IGNITE-13805 Code cleanup. --- .../processors/cache/ClusterCachesInfo.java | 2 +- .../processors/cache/GridCacheProcessor.java | 2 +- .../snapshot/IgniteSnapshotManager.java | 10 +++++----- .../snapshot/SnapshotRestoreProcess.java | 14 ++++++-------- .../IgniteClusterSnapshotRestoreSelfTest.java | 2 +- 5 files changed, 14 insertions(+), 16 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index ead0c55481b1eb..24606e7ca87e57 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1030,7 +1030,7 @@ else if (encMgr.masterKeyDigest() != null && if (err == null && !req.restoredCache()) { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); - if (snapshotMgr.isCacheRestoring(cacheName, ccfg.getGroupName())) { + if (snapshotMgr.isRestoring(cacheName, ccfg.getGroupName())) { err = new IgniteCheckedException("Cache start failed. A cache or group with the same name is " + "currently being restored from a snapshot [cache=" + cacheName + (ccfg.getGroupName() == null ? "" : ", group=" + ccfg.getGroupName()) + ']'); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 5d48a9c91c0caf..39b1a57f44a6d5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -4346,7 +4346,7 @@ private IgniteNodeValidationResult validateRestartingCaches(ClusterNode node) { * @return Node validation result if there was an issue with the joining node, {@code null} otherwise. */ private IgniteNodeValidationResult validateRestoringCaches(ClusterNode node) { - if (ctx.cache().context().snapshotMgr().isSnapshotRestoring()) { + if (ctx.cache().context().snapshotMgr().isRestoring()) { String msg = "Joining node during caches restore is not allowed [joiningNodeId=" + node.id() + ']'; return new IgniteNodeValidationResult(node.id(), msg); 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 fb56874e175827..001a30ac6c3642 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 @@ -772,8 +772,8 @@ public boolean isSnapshotCreating() { * * @return {@code True} if the snapshot restore operation is in progress. */ - public boolean isSnapshotRestoring() { - return restoreCacheGrpProc.isSnapshotRestoring(); + public boolean isRestoring() { + return restoreCacheGrpProc.isRestoring(); } /** @@ -783,8 +783,8 @@ public boolean isSnapshotRestoring() { * @param grpName Cache group name. * @return {@code True} if the cache or group with the specified name is being restored. */ - public boolean isCacheRestoring(String cacheName, @Nullable String grpName) { - return restoreCacheGrpProc.isCacheRestoring(cacheName, grpName); + public boolean isRestoring(String cacheName, @Nullable String grpName) { + return restoreCacheGrpProc.isRestoring(cacheName, grpName); } /** @@ -1042,7 +1042,7 @@ public List readSnapshotMetadatas(String snpName) { if (!clusterState.hasBaselineTopology()) throw new IgniteException("Snapshot operation has been rejected. The baseline topology is not configured for cluster."); - if (isSnapshotRestoring()) { + if (isRestoring()) { throw new IgniteException("Snapshot operation has been rejected. " + "Cache group restore operation is currently in progress."); } 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 61aa4ae0db1fab..9747b50230b7ba 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 @@ -234,7 +234,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * * @return {@code True} if the snapshot restore operation is in progress. */ - public boolean isSnapshotRestoring() { + public boolean isRestoring() { return opCtx != null; } @@ -245,7 +245,7 @@ public boolean isSnapshotRestoring() { * @param grpName Cache group name. * @return {@code True} if the cache or group with the specified name is currently being restored. */ - public boolean isCacheRestoring(String cacheName, @Nullable String grpName) { + public boolean isRestoring(String cacheName, @Nullable String grpName) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) @@ -261,17 +261,15 @@ public boolean isCacheRestoring(String cacheName, @Nullable String grpName) { for (File grpDir : opCtx0.dirs) { String locGrpName = FilePageStoreManager.cacheGroupName(grpDir); - if (grpName == null) { - if (CU.cacheId(locGrpName) == cacheId) - return true; - } - else { + if (grpName != null) { if (cacheName.equals(locGrpName)) return true; if (CU.cacheId(locGrpName) == CU.cacheId(grpName)) return true; } + else if (CU.cacheId(locGrpName) == cacheId) + return true; } return false; @@ -496,7 +494,7 @@ private IgniteInternalFuture> restoreAsync( * @throws IgniteCheckedException If failed. */ private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) throws IgniteCheckedException { - if (isSnapshotRestoring()) { + if (isRestoring()) { throw new IgniteCheckedException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); } 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 94359f0dc6d403..5a3c7ce6812ec3 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 @@ -670,7 +670,7 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh assertNull("nodeIdx=" + nodeIdx + ", cache=" + cacheName, desc); GridTestUtils.waitForCondition( - () -> !grid.context().cache().context().snapshotMgr().isSnapshotRestoring(), + () -> !grid.context().cache().context().snapshotMgr().isRestoring(), TIMEOUT); File dir = resolveCacheDir(grid, cacheName); From 72984be831c8e981d54b6f2505c151f5190b014e Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 19 Mar 2021 08:12:38 +0300 Subject: [PATCH 47/98] IGNITE-13805 Simplify restoreAsync (exp). --- .../snapshot/SnapshotRestoreProcess.java | 88 ++++++++++--------- .../IgniteClusterSnapshotRestoreSelfTest.java | 4 +- 2 files changed, 50 insertions(+), 42 deletions(-) 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 9747b50230b7ba..ba4aa015f8c3bf 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 @@ -35,6 +35,7 @@ import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BooleanSupplier; +import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -389,8 +390,36 @@ private IgniteInternalFuture> prepare(SnapshotRestore ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); } - return restoreAsync(opCtx0, ctx.localNodeId().equals(req.updateMetaNodeId())); + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); + BooleanSupplier stopChecker = () -> { + if (opCtx.err.get() != null) + return true; + + if (Thread.currentThread().isInterrupted()) { + errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); + + return true; + } + + return false; + }; + GridFutureAdapter> retFut = new GridFutureAdapter<>(); + + restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { + Throwable err = opCtx.err.get(); + + if (err != null) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + + retFut.onDone(err); + } else + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + }); + + return retFut; } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { log.error("Unable to restore cache group(s) from the snapshot " + "[requestID=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); @@ -402,49 +431,40 @@ private IgniteInternalFuture> prepare(SnapshotRestore /** * Copy partition files and update binary metadata. * - * @param opCtx Snapshot restore operation context. + * @param snpName Snapshot name. + * @param dirs Cache directories to restore from the snapshot. * @param updateMeta Update binary metadata flag. + * @param stopChecker Prcoess interrupt checker. + * @param errHnd Error handler. * @throws IgniteCheckedException If failed. */ - private IgniteInternalFuture> restoreAsync( - SnapshotRestoreContext opCtx, - boolean updateMeta + private CompletableFuture restoreAsync( + String snpName, + Collection dirs, + boolean updateMeta, + BooleanSupplier stopChecker, + Consumer errHnd ) throws IgniteCheckedException { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); - BooleanSupplier stopChecker = () -> { - if (opCtx.err.get() != null) - return true; - - if (Thread.currentThread().isInterrupted()) { - opCtx.err.compareAndSet(null, new IgniteInterruptedCheckedException("Thread has been interrupted.")); - - return true; - } - - return false; - }; - - GridFutureAdapter> retFut = new GridFutureAdapter<>(); - List> futs = new ArrayList<>(); if (updateMeta) { - File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(opCtx.snpName).getAbsolutePath(), pdsFolderName); + File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), pdsFolderName); futs.add(CompletableFuture.runAsync(() -> { try { ctx.cacheObjects().updateMetadata(binDir, stopChecker); } catch (IgniteCheckedException e) { - opCtx.err.compareAndSet(null, e); + errHnd.accept(e); } }, snapshotMgr.snapshotExecutorService())); } - for (File cacheDir : this.opCtx.dirs) { - File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(this.opCtx.snpName), + for (File cacheDir : dirs) { + File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(snpName), Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); assert snpCacheDir.exists() : "node=" + ctx.localNodeId() + ", dir=" + snpCacheDir; @@ -458,7 +478,7 @@ private IgniteInternalFuture> restoreAsync( if (log.isDebugEnabled()) { log.debug("Copying file from the snapshot " + - "[snapshot=" + this.opCtx.snpName + + "[snapshot=" + snpName + ", src=" + snpFile + ", target=" + target + "]"); } @@ -467,25 +487,13 @@ private IgniteInternalFuture> restoreAsync( Files.copy(snpFile.toPath(), target.toPath()); } catch (IOException e) { - opCtx.err.compareAndSet(null, e); + errHnd.accept(e); } }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); } } - CompletableFuture.allOf(futs.toArray(new CompletableFuture[0])).thenAccept(res -> { - Throwable err = opCtx.err.get(); - - if (err != null) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); - - retFut.onDone(err); - } else - retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); - }); - - return retFut; + return CompletableFuture.allOf(futs.toArray(new CompletableFuture[0])); } /** @@ -690,7 +698,7 @@ private Exception checNodeLeft(Set reqNodes, Set respNodes) { leftNodes.removeAll(respNodes); - return new IgniteCheckedException(OP_REJECT_MSG + + return new ClusterTopologyCheckedException(OP_REJECT_MSG + "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); } 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 5a3c7ce6812ec3..d3829ab3fc48b6 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 @@ -505,8 +505,8 @@ private void checkTopologyChange(boolean stopNode) throws Exception { GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(TIMEOUT), - IgniteException.class, - "Cache group restore operation was rejected. Server node(s) has left the cluster" + ClusterTopologyCheckedException.class, + "Server node(s) has left the cluster" ); ensureCacheDirEmpty(3, dfltCacheCfg.getName()); From 295a446db213774ea7d49826e19edddc58fb7198 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 19 Mar 2021 13:05:22 +0300 Subject: [PATCH 48/98] IGNITE-13805 Remove redundant rollback. --- .../snapshot/SnapshotRestoreProcess.java | 45 ++++++++----------- 1 file changed, 19 insertions(+), 26 deletions(-) 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 ba4aa015f8c3bf..ea9fbd91b08f01 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 @@ -553,31 +553,6 @@ else if (cacheDir.list().length > 0) { return new SnapshotRestoreContext(req, cacheDirs, cfgsById); } - /** - * Rollback changes made by process. - * - * @param opCtx Snapshot restore operation context. - */ - private void rollback(@Nullable SnapshotRestoreContext opCtx) { - if (opCtx == null || F.isEmpty(opCtx.dirs)) - return; - - if (log.isInfoEnabled()) { - log.info("Performing local rollback routine for restored cache groups " + - "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']'); - } - - for (File cacheDir : opCtx.dirs) { - if (!cacheDir.exists()) - continue; - - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); - - U.delete(cacheDir); - } - } - /** * @param reqId Request ID. * @param res Results. @@ -713,7 +688,25 @@ private IgniteInternalFuture rollback(UUID reqId) { if (ctx.clientNode()) return new GridFinishedFuture<>(); - rollback(opCtx); + SnapshotRestoreContext opCtx0 = opCtx; + + if (F.isEmpty(opCtx0.dirs)) + return new GridFinishedFuture<>(); + + if (log.isInfoEnabled()) { + log.info("Performing local rollback routine for restored cache groups " + + "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); + } + + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; + + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); + + U.delete(cacheDir); + } return new GridFinishedFuture<>(true); } From c5d07918b65a97cdb35fe14424fa1d6213a3c236 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 19 Mar 2021 13:17:24 +0300 Subject: [PATCH 49/98] IGNITE-13805 (minor) Code cleanup. --- .../CacheObjectBinaryProcessorImpl.java | 30 +++++---------- .../snapshot/SnapshotRestoreProcess.java | 38 ++++++++++--------- 2 files changed, 31 insertions(+), 37 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index ce1a2cb1c90ab7..dd94353d8e05f6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -25,7 +25,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -996,9 +995,18 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { /** {@inheritDoc} */ @Override public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException { - Collection metadata = readMetadata(metadataDir); + if (!metadataDir.exists()) + return; try { + ConcurrentMap metaCache = new ConcurrentHashMap<>(); + + BinaryMetadataFileStore binaryMetaFileStore = new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir); + + binaryMetaFileStore.restoreMetadata(); + + Collection metadata = F.viewReadOnly(metaCache.values(), BinaryMetadataHolder::metadata); + // Check the compatibility of the binary metadata files stored in the specified directory. for (BinaryMetadata newMeta : metadata) { BinaryMetadata oldMeta = binaryMetadata(newMeta.typeId()); @@ -1019,24 +1027,6 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { } } - /** - * @param metadataDir Directory containing binary metadata files. - * @return List of found metadata types. - * @throws IgniteCheckedException If failed. - */ - private Collection readMetadata(File metadataDir) throws IgniteCheckedException { - if (!metadataDir.exists()) - return Collections.emptyList(); - - ConcurrentMap metaCache = new ConcurrentHashMap<>(); - - BinaryMetadataFileStore binaryMetaFileStore = new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir); - - binaryMetaFileStore.restoreMetadata(); - - return F.viewReadOnly(metaCache.values(), BinaryMetadataHolder::metadata); - } - /** {@inheritDoc} */ @Override public BinaryObject buildEnum(String typeName, int ord) throws BinaryObjectException { A.notNullOrEmpty(typeName, "enum type name"); 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 ea9fbd91b08f01..423b577f8e368d 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 @@ -29,6 +29,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -293,11 +294,11 @@ private void finishProcess(@Nullable Throwable err) { if (err != null) { log.error("Failed to restore snapshot cache group" + (opCtx0 == null ? "" : - " [requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'), err); + " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'), err); } else if (log.isInfoEnabled()) { log.info("Successfully restored cache group(s) from the snapshot" + (opCtx0 == null ? "" : - " [requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']')); + " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']')); } GridFutureAdapter fut0 = fut; @@ -386,7 +387,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore } if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [requestID=" + req.requestId() + + log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); } @@ -412,7 +413,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (err != null) { log.error("Unable to restore cache group(s) from the snapshot " + - "[requestID=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); retFut.onDone(err); } else @@ -422,7 +423,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore return retFut; } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { log.error("Unable to restore cache group(s) from the snapshot " + - "[requestID=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); + "[reqId=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); return new GridFinishedFuture<>(e); } @@ -493,7 +494,9 @@ private CompletableFuture restoreAsync( } } - return CompletableFuture.allOf(futs.toArray(new CompletableFuture[0])); + int futsSize = futs.size(); + + return CompletableFuture.allOf(futs.toArray(new CompletableFuture[futsSize])); } /** @@ -613,21 +616,24 @@ private IgniteInternalFuture cacheStart(UUID reqId) { SnapshotRestoreContext opCtx0 = opCtx; - assert opCtx0 != null : ctx.localNodeId(); - - if (!U.isLocalNodeCoordinator(ctx.discovery())) - return new GridFinishedFuture<>(); + if (opCtx0 == null) { + return new GridFinishedFuture<>(new IgniteIllegalStateException("Context has not been created on server " + + "node during prepare operation [reqId=" + reqId + ", nodeId=" + ctx.localNodeId() + ']')); + } Throwable err = opCtx0.err.get(); if (err != null) return new GridFinishedFuture<>(err); + if (!U.isLocalNodeCoordinator(ctx.discovery())) + return new GridFinishedFuture<>(); + Collection ccfgs = opCtx0.cfgs.values(); if (log.isInfoEnabled()) { log.info("Starting restored caches " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + + "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ", caches=" + F.viewReadOnly(ccfgs, c -> c.config().getName()) + ']'); } @@ -645,10 +651,8 @@ private void finishCacheStart(UUID reqId, Map res, Map rollback(UUID reqId) { if (log.isInfoEnabled()) { log.info("Performing local rollback routine for restored cache groups " + - "[requestID=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); + "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); } for (File cacheDir : opCtx0.dirs) { @@ -728,7 +732,7 @@ private void finishRollback(UUID reqId, Map res, Map reqNodes, Set respNodes) { leftNodes.removeAll(respNodes); return new ClusterTopologyCheckedException(OP_REJECT_MSG + - "Server node(s) has left the cluster [nodeId=" + leftNodes + ']'); + "Required node has left the cluster [nodeId=" + leftNodes + ']'); } return null; 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 d3829ab3fc48b6..3f0f4a6835b4c9 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 @@ -506,7 +506,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, - "Server node(s) has left the cluster" + "Required node has left the cluster" ); ensureCacheDirEmpty(3, dfltCacheCfg.getName()); From cbc0cbb3b7c673c1511a7cd60d5d9b7f04469be2 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 22 Mar 2021 18:03:54 +0300 Subject: [PATCH 51/98] IGNITE-13805 (minor) code cleanup. --- .../cache/binary/CacheObjectBinaryProcessorImpl.java | 7 +++---- .../cache/persistence/snapshot/SnapshotRestoreProcess.java | 3 +-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index dd94353d8e05f6..1d2e6ddfc5f36d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -1001,13 +1001,12 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { try { ConcurrentMap metaCache = new ConcurrentHashMap<>(); - BinaryMetadataFileStore binaryMetaFileStore = new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir); - - binaryMetaFileStore.restoreMetadata(); + new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir) + .restoreMetadata(); Collection metadata = F.viewReadOnly(metaCache.values(), BinaryMetadataHolder::metadata); - // Check the compatibility of the binary metadata files stored in the specified directory. + // Check the compatibility of the binary metadata. for (BinaryMetadata newMeta : metadata) { BinaryMetadata oldMeta = binaryMetadata(newMeta.typeId()); 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 85a9303b45e8d8..9234c63acb9e03 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 @@ -26,7 +26,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -173,7 +172,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - Set dataNodes = new LinkedHashSet<>(); + Set dataNodes = new HashSet<>(); Map> metas = f.result(); Map reqGrpIds = cacheGrpNames.stream().collect(Collectors.toMap(CU::cacheId, v -> v)); From 2d72c13e3f85bbab85b239c122922c62503e8138 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 23 Mar 2021 10:00:06 +0300 Subject: [PATCH 52/98] IGNITE-13805 Code cleanup. --- .../snapshot/SnapshotRestoreProcess.java | 82 ++++++++++--------- ...quest.java => SnapshotRestoreRequest.java} | 6 +- .../IgniteCacheObjectProcessor.java | 2 +- 3 files changed, 47 insertions(+), 43 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestorePrepareRequest.java => SnapshotRestoreRequest.java} (94%) 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 9234c63acb9e03..408415abd1386c 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 @@ -28,7 +28,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -83,7 +82,7 @@ public class SnapshotRestoreProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess> prepareRestoreProc; + private final DistributedProcess> prepareRestoreProc; /** Cache group restore cache start phase. */ private final DistributedProcess cacheStartProc; @@ -131,17 +130,6 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames new IgniteException(OP_REJECT_MSG + "Client and daemon nodes can not perform this operation.")); } - synchronized (this) { - GridFutureAdapter fut0 = fut; - - if (opCtx != null || (fut0 != null && !fut0.isDone())) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); - } - - fut = new GridFutureAdapter<>(); - } - DiscoveryDataClusterState clusterState = ctx.state().clusterState(); if (clusterState.state() != ClusterState.ACTIVE || clusterState.transition()) @@ -152,6 +140,11 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames new IgniteException(OP_REJECT_MSG + "The baseline topology is not configured for cluster.")); } + if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) { + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation.")); + } + IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); if (snpMgr.isSnapshotCreating()) { @@ -159,9 +152,15 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); } - if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation.")); + synchronized (this) { + GridFutureAdapter fut0 = fut; + + if (opCtx != null || (fut0 != null && !fut0.isDone())) { + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); + } + + fut = new GridFutureAdapter<>(); } snpMgr.collectSnapshotMetadata(snpName).listen( @@ -216,7 +215,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - SnapshotRestorePrepareRequest req = new SnapshotRestorePrepareRequest(UUID.randomUUID(), + SnapshotRestoreRequest req = new SnapshotRestoreRequest(UUID.randomUUID(), snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); prepareRestoreProc.start(req.requestId(), req); @@ -234,7 +233,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * @return {@code True} if the snapshot restore operation is in progress. */ public boolean isRestoring() { - return opCtx != null; + GridFutureAdapter fut0; + + return opCtx != null || ((fut0 = fut) != null && !fut0.isDone()); } /** @@ -286,24 +287,22 @@ private void finishProcess() { * * @param err Error, if any. */ - private void finishProcess(@Nullable Throwable err) { + private void finishProcess(@Nullable Exception err) { SnapshotRestoreContext opCtx0 = opCtx; - if (err != null) { - log.error("Failed to restore snapshot cache group" + (opCtx0 == null ? "" : - " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'), err); - } - else if (log.isInfoEnabled()) { - log.info("Successfully restored cache group(s) from the snapshot" + (opCtx0 == null ? "" : - " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']')); - } + String details = opCtx0 == null ? "" : " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'; + + if (err != null) + log.error("Failed to restore snapshot cache group" + details, err); + else if (log.isInfoEnabled()) + log.info("Successfully restored cache group(s) from the snapshot" + details); + + opCtx = null; GridFutureAdapter fut0 = fut; if (fut0 != null) fut0.onDone(null, err); - - opCtx = null; } /** @@ -350,7 +349,7 @@ private void ensureCacheAbsent(String name) { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture> prepare(SnapshotRestorePrepareRequest req) { + private IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -360,6 +359,9 @@ private IgniteInternalFuture> prepare(SnapshotRestore if (state.state() != ClusterState.ACTIVE || state.transition()) throw new IgniteCheckedException(OP_REJECT_MSG + "The cluster should be active."); + if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) + throw new IgniteCheckedException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); + for (UUID nodeId : req.nodes()) { ClusterNode node = ctx.discovery().node(nodeId); @@ -434,7 +436,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore * @param snpName Snapshot name. * @param dirs Cache directories to restore from the snapshot. * @param updateMeta Update binary metadata flag. - * @param stopChecker Prcoess interrupt checker. + * @param stopChecker Process interrupt checker. * @param errHnd Error handler. * @throws IgniteCheckedException If failed. */ @@ -503,8 +505,8 @@ private CompletableFuture restoreAsync( * @return Snapshot restore operation context. * @throws IgniteCheckedException If failed. */ - private SnapshotRestoreContext prepareContext(SnapshotRestorePrepareRequest req) throws IgniteCheckedException { - if (isRestoring()) { + private SnapshotRestoreContext prepareContext(SnapshotRestoreRequest req) throws IgniteCheckedException { + if (opCtx != null) { throw new IgniteCheckedException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); } @@ -577,7 +579,7 @@ private void finishPrepare(UUID reqId, Map> res } if (failure == null) - failure = checNodeLeft(opCtx0.nodes, res.keySet()); + failure = checkNodeLeft(opCtx0.nodes, res.keySet()); // Context has been created - should rollback changes cluster-wide. if (failure != null) { @@ -636,6 +638,8 @@ private IgniteInternalFuture cacheStart(UUID reqId) { ", caches=" + F.viewReadOnly(ccfgs, c -> c.config().getName()) + ']'); } + // We set the topology node IDs required to successfully start the cache, if any of the required nodes leave + // the cluster during the cache startup, the whole procedure will be rolled back. return ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes); } @@ -650,8 +654,8 @@ private void finishCacheStart(UUID reqId, Map res, Map res, Map reqNodes, Set respNodes) { + private Exception checkNodeLeft(Set reqNodes, Set respNodes) { if (!respNodes.containsAll(reqNodes)) { Set leftNodes = new HashSet<>(reqNodes); @@ -754,7 +758,7 @@ private static class SnapshotRestoreContext { private final Collection dirs; /** The exception that led to the interruption of the process. */ - private final AtomicReference err = new AtomicReference<>(); + private final AtomicReference err = new AtomicReference<>(); /** Cache ID to configuration mapping. */ private volatile Map cfgs; @@ -764,7 +768,7 @@ private static class SnapshotRestoreContext { * @param dirs List of cache group names to restore from the snapshot. * @param cfgs Cache ID to configuration mapping. */ - protected SnapshotRestoreContext(SnapshotRestorePrepareRequest req, Collection dirs, + protected SnapshotRestoreContext(SnapshotRestoreRequest req, Collection dirs, Map cfgs) { reqId = req.requestId(); snpName = req.snapshotName(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java similarity index 94% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java index 0acc3b404aeb07..98d805f243e3f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestorePrepareRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java @@ -26,7 +26,7 @@ /** * Request to prepare cache group restore from the snapshot. */ -public class SnapshotRestorePrepareRequest implements Serializable { +public class SnapshotRestoreRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -52,7 +52,7 @@ public class SnapshotRestorePrepareRequest implements Serializable { * @param grps List of cache group names to restore from the snapshot. * @param updateMetaNodeId Node ID from which to update the binary metadata. */ - public SnapshotRestorePrepareRequest( + public SnapshotRestoreRequest( UUID reqId, String snpName, Set nodes, @@ -103,6 +103,6 @@ public UUID updateMetaNodeId() { /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SnapshotRestorePrepareRequest.class, this); + return S.toString(SnapshotRestoreRequest.class, this); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java index a4c9eb4fc4a1dd..170a3c09c28451 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java @@ -311,7 +311,7 @@ public void updateMetadata(int typeId, String typeName, @Nullable String affKeyF * Merge the binary metadata files stored in the specified directory. * * @param metadataDir Directory containing binary metadata files. - * @param stopChecker Prcoess interrupt checker. + * @param stopChecker Process interrupt checker. * @throws IgniteCheckedException If failed. */ public void updateMetadata(File metadataDir, BooleanSupplier stopChecker) throws IgniteCheckedException; From df469ca6b621160cdd2fad92bfb97afaf199b60c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 23 Mar 2021 15:52:43 +0300 Subject: [PATCH 53/98] IGNITE-13805 Sync diff snapshot ops startup. --- .../snapshot/IgniteSnapshotManager.java | 10 ++++------ .../snapshot/SnapshotRestoreProcess.java | 19 ++++++++++++------- 2 files changed, 16 insertions(+), 13 deletions(-) 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 001a30ac6c3642..6a381931f6b197 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 @@ -320,7 +320,7 @@ public IgniteSnapshotManager(GridKernalContext ctx) { marsh = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName()); - restoreCacheGrpProc = new SnapshotRestoreProcess(ctx); + restoreCacheGrpProc = new SnapshotRestoreProcess(ctx, snpOpMux); } /** @@ -1042,11 +1042,6 @@ public List readSnapshotMetadatas(String snpName) { if (!clusterState.hasBaselineTopology()) throw new IgniteException("Snapshot operation has been rejected. The baseline topology is not configured for cluster."); - if (isRestoring()) { - throw new IgniteException("Snapshot operation has been rejected. " + - "Cache group restore operation is currently in progress."); - } - if (cctx.kernalContext().clientNode()) { ClusterNode crd = U.oldest(cctx.kernalContext().discovery().aliveServerNodes(), null); @@ -1074,6 +1069,9 @@ public List readSnapshotMetadatas(String snpName) { if (localSnapshotNames().contains(name)) throw new IgniteException("Create snapshot request has been rejected. Snapshot with given name already exists on local node."); + if (isRestoring()) + throw new IgniteException("Snapshot operation has been rejected. Cache group restore operation is currently in progress."); + snpFut0 = new ClusterSnapshotFuture(UUID.randomUUID(), name); clusterSnpFut = snpFut0; 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 408415abd1386c..e6bc716a8f8585 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 @@ -90,6 +90,9 @@ public class SnapshotRestoreProcess { /** Cache group restore rollback phase. */ private final DistributedProcess rollbackRestoreProc; + /** Mutex used to order cluster snapshot operation progress. */ + private final Object snpOpMux; + /** Logger. */ private final IgniteLogger log; @@ -101,9 +104,11 @@ public class SnapshotRestoreProcess { /** * @param ctx Kernal context. + * @param snpOpMux Mutex used to order cluster snapshot operation progress. */ - public SnapshotRestoreProcess(GridKernalContext ctx) { + public SnapshotRestoreProcess(GridKernalContext ctx, Object snpOpMux) { this.ctx = ctx; + this.snpOpMux = snpOpMux; log = ctx.log(getClass()); @@ -147,12 +152,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); - if (snpMgr.isSnapshotCreating()) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); - } - - synchronized (this) { + synchronized (snpOpMux) { GridFutureAdapter fut0 = fut; if (opCtx != null || (fut0 != null && !fut0.isDone())) { @@ -160,6 +160,11 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); } + if (snpMgr.isSnapshotCreating()) { + return new IgniteFinishedFutureImpl<>( + new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); + } + fut = new GridFutureAdapter<>(); } From 63256dcf727863004e61d44230bbf78573d1e8cd Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 23 Mar 2021 15:58:02 +0300 Subject: [PATCH 54/98] IGNITE-13805 (minor) Code cleanup. --- .../snapshot/SnapshotRestoreProcess.java | 50 ++++++++----------- 1 file changed, 21 insertions(+), 29 deletions(-) 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 e6bc716a8f8585..3c0707f9fb1e73 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 @@ -130,45 +130,37 @@ public SnapshotRestoreProcess(GridKernalContext ctx, Object snpOpMux) { * @return Future that will be completed when the restore operation is complete and the cache groups are started. */ public IgniteFuture start(String snpName, Collection cacheGrpNames) { - if (ctx.clientNode()) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "Client and daemon nodes can not perform this operation.")); - } + try { + if (ctx.clientNode()) + throw new IgniteException(OP_REJECT_MSG + "Client and daemon nodes can not perform this operation."); - DiscoveryDataClusterState clusterState = ctx.state().clusterState(); + DiscoveryDataClusterState clusterState = ctx.state().clusterState(); - if (clusterState.state() != ClusterState.ACTIVE || clusterState.transition()) - return new IgniteFinishedFutureImpl<>(new IgniteException(OP_REJECT_MSG + "The cluster should be active.")); + if (clusterState.state() != ClusterState.ACTIVE || clusterState.transition()) + throw new IgniteException(OP_REJECT_MSG + "The cluster should be active."); - if (!clusterState.hasBaselineTopology()) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "The baseline topology is not configured for cluster.")); - } + if (!clusterState.hasBaselineTopology()) + throw new IgniteException(OP_REJECT_MSG + "The baseline topology is not configured for cluster."); - if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation.")); - } + if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) + throw new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation."); - IgniteSnapshotManager snpMgr = ctx.cache().context().snapshotMgr(); + synchronized (snpOpMux) { + GridFutureAdapter fut0 = fut; - synchronized (snpOpMux) { - GridFutureAdapter fut0 = fut; + if (opCtx != null || (fut0 != null && !fut0.isDone())) + throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - if (opCtx != null || (fut0 != null && !fut0.isDone())) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed.")); - } + if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) + throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); - if (snpMgr.isSnapshotCreating()) { - return new IgniteFinishedFutureImpl<>( - new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress.")); + fut = new GridFutureAdapter<>(); } - - fut = new GridFutureAdapter<>(); + } catch (IgniteException e) { + return new IgniteFinishedFutureImpl<>(e); } - snpMgr.collectSnapshotMetadata(snpName).listen( + ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { fut.onDone(f.error()); @@ -200,7 +192,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - snpMgr.runSnapshotVerfification(metas).listen( + ctx.cache().context().snapshotMgr().runSnapshotVerfification(metas).listen( f0 -> { if (f0.error() != null) { fut.onDone(f0.error()); From 67c4cf9042bf12f0e163f64ee8b07991d75fb45c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 23 Mar 2021 17:19:13 +0300 Subject: [PATCH 55/98] IGNITE-13805 Don;t sync snapshot oper startup. --- .../snapshot/IgniteSnapshotManager.java | 2 +- .../snapshot/SnapshotRestoreProcess.java | 23 +++++++------------ 2 files changed, 9 insertions(+), 16 deletions(-) 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 6a381931f6b197..2a8883c6c4f8b7 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 @@ -320,7 +320,7 @@ public IgniteSnapshotManager(GridKernalContext ctx) { marsh = MarshallerUtils.jdkMarshaller(ctx.igniteInstanceName()); - restoreCacheGrpProc = new SnapshotRestoreProcess(ctx, snpOpMux); + restoreCacheGrpProc = new SnapshotRestoreProcess(ctx); } /** 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 3c0707f9fb1e73..4cd96aa45507c2 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 @@ -90,9 +90,6 @@ public class SnapshotRestoreProcess { /** Cache group restore rollback phase. */ private final DistributedProcess rollbackRestoreProc; - /** Mutex used to order cluster snapshot operation progress. */ - private final Object snpOpMux; - /** Logger. */ private final IgniteLogger log; @@ -104,11 +101,9 @@ public class SnapshotRestoreProcess { /** * @param ctx Kernal context. - * @param snpOpMux Mutex used to order cluster snapshot operation progress. */ - public SnapshotRestoreProcess(GridKernalContext ctx, Object snpOpMux) { + public SnapshotRestoreProcess(GridKernalContext ctx) { this.ctx = ctx; - this.snpOpMux = snpOpMux; log = ctx.log(getClass()); @@ -145,15 +140,15 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames if (!IgniteFeatures.allNodesSupports(ctx.grid().cluster().nodes(), SNAPSHOT_RESTORE_CACHE_GROUP)) throw new IgniteException(OP_REJECT_MSG + "Not all nodes in the cluster support restore operation."); - synchronized (snpOpMux) { + if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) + throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); + + synchronized (this) { GridFutureAdapter fut0 = fut; - if (opCtx != null || (fut0 != null && !fut0.isDone())) + if (isRestoring() || (fut0 != null && !fut0.isDone())) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - if (ctx.cache().context().snapshotMgr().isSnapshotCreating()) - throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); - fut = new GridFutureAdapter<>(); } } catch (IgniteException e) { @@ -230,9 +225,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * @return {@code True} if the snapshot restore operation is in progress. */ public boolean isRestoring() { - GridFutureAdapter fut0; - - return opCtx != null || ((fut0 = fut) != null && !fut0.isDone()); + return opCtx != null; } /** @@ -503,7 +496,7 @@ private CompletableFuture restoreAsync( * @throws IgniteCheckedException If failed. */ private SnapshotRestoreContext prepareContext(SnapshotRestoreRequest req) throws IgniteCheckedException { - if (opCtx != null) { + if (isRestoring()) { throw new IgniteCheckedException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); } From db3f917e5d411158346603cbf6c012f777e29a98 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 23 Mar 2021 17:25:27 +0300 Subject: [PATCH 56/98] IGNITE-13805 Complete user future in diff thread pool. --- .../snapshot/SnapshotRestoreProcess.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) 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 4cd96aa45507c2..b06739dd009805 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 @@ -144,9 +144,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); synchronized (this) { - GridFutureAdapter fut0 = fut; - - if (isRestoring() || (fut0 != null && !fut0.isDone())) + if (isRestoring()) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); fut = new GridFutureAdapter<>(); @@ -158,7 +156,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - fut.onDone(f.error()); + finishProcess(f.error()); return; } @@ -181,7 +179,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + finishProcess(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); return; @@ -202,7 +200,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - fut.onDone(new IgniteException(sb.toString())); + finishProcess(new IgniteException(sb.toString())); return; } @@ -225,7 +223,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * @return {@code True} if the snapshot restore operation is in progress. */ public boolean isRestoring() { - return opCtx != null; + return opCtx != null || fut != null; } /** @@ -277,7 +275,7 @@ private void finishProcess() { * * @param err Error, if any. */ - private void finishProcess(@Nullable Exception err) { + private void finishProcess(@Nullable Throwable err) { SnapshotRestoreContext opCtx0 = opCtx; String details = opCtx0 == null ? "" : " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'; @@ -291,8 +289,11 @@ else if (log.isInfoEnabled()) GridFutureAdapter fut0 = fut; - if (fut0 != null) - fut0.onDone(null, err); + if (fut0 != null) { + fut = null; + + ctx.getSystemExecutorService().submit(() -> fut0.onDone(null, err)); + } } /** @@ -496,7 +497,7 @@ private CompletableFuture restoreAsync( * @throws IgniteCheckedException If failed. */ private SnapshotRestoreContext prepareContext(SnapshotRestoreRequest req) throws IgniteCheckedException { - if (isRestoring()) { + if (opCtx != null) { throw new IgniteCheckedException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); } From 18b60d54453b96b02ae19cafd7b158ec838bff21 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 24 Mar 2021 13:26:31 +0300 Subject: [PATCH 57/98] IGNITE-13805 Move rollback in diff thread. --- .../snapshot/SnapshotRestoreProcess.java | 30 +++++++++++-------- 1 file changed, 18 insertions(+), 12 deletions(-) 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 b06739dd009805..af1cdcdbf81850 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 @@ -691,22 +691,28 @@ private IgniteInternalFuture rollback(UUID reqId) { if (F.isEmpty(opCtx0.dirs)) return new GridFinishedFuture<>(); - if (log.isInfoEnabled()) { - log.info("Performing local rollback routine for restored cache groups " + - "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); - } + GridFutureAdapter retFut = new GridFutureAdapter<>(); - for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + if (log.isInfoEnabled()) { + log.info("Performing local rollback routine for restored cache groups " + + "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); + } - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; - U.delete(cacheDir); - } + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); + + U.delete(cacheDir); + } + + retFut.onDone(true); + }); - return new GridFinishedFuture<>(true); + return retFut; } /** From 43bd1d6d862ccdacb5afb3c8e43d33f1de238633 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 24 Mar 2021 16:41:28 +0300 Subject: [PATCH 58/98] IGNITE-13805 Graceful shutdown (wip). --- .../snapshot/SnapshotRestoreProcess.java | 96 +++++++++++++++---- 1 file changed, 77 insertions(+), 19 deletions(-) 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 af1cdcdbf81850..3a75610471fdf3 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 @@ -46,12 +46,14 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; +import org.apache.ignite.internal.util.GridBusyLock; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -93,6 +95,9 @@ public class SnapshotRestoreProcess { /** Logger. */ private final IgniteLogger log; + /** Busy lock. */ + private final GridBusyLock busyLock = new GridBusyLock(); + /** The future to be completed when the cache restore process is complete. */ private volatile GridFutureAdapter fut; @@ -318,8 +323,24 @@ public void onNodeLeft(UUID leftNodeId) { public void stop(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 != null) - opCtx0.err.compareAndSet(null, reason); + if (opCtx0 == null) + return; + + opCtx0.err.compareAndSet(null, reason); + + busyLock.block(); + + try { + IgniteInternalFuture stopFut = opCtx0.stopFut; + + if (stopFut != null && !stopFut.isDone()) + stopFut.get(); + + } catch (IgniteCheckedException ignore) { + // No-op. + } finally { + busyLock.unblock(); + } } /** @@ -369,6 +390,9 @@ private IgniteInternalFuture> prepare(SnapshotRestore SnapshotRestoreContext opCtx0 = opCtx; + if (ctx.isStopping()) + throw new NodeStoppingException("Node is stopping."); + if (opCtx0.dirs.isEmpty()) return new GridFinishedFuture<>(); @@ -400,6 +424,8 @@ private IgniteInternalFuture> prepare(SnapshotRestore GridFutureAdapter> retFut = new GridFutureAdapter<>(); + opCtx0.stopFut = retFut; + restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { Throwable err = opCtx.err.get(); @@ -447,11 +473,19 @@ private CompletableFuture restoreAsync( File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), pdsFolderName); futs.add(CompletableFuture.runAsync(() -> { + if (!busyLock.enterBusy()) { + errHnd.accept(new IgniteCheckedException("Node has been stopped or deactivated.")); + + return; + } + try { ctx.cacheObjects().updateMetadata(binDir, stopChecker); } catch (IgniteCheckedException e) { errHnd.accept(e); + } finally { + busyLock.leaveBusy(); } }, snapshotMgr.snapshotExecutorService())); } @@ -467,21 +501,30 @@ private CompletableFuture restoreAsync( if (stopChecker.getAsBoolean()) return; - File target = new File(cacheDir, snpFile.getName()); + if (!busyLock.enterBusy()) { + errHnd.accept(new IgniteCheckedException("Node has been stopped or deactivated.")); - if (log.isDebugEnabled()) { - log.debug("Copying file from the snapshot " + - "[snapshot=" + snpName + - ", src=" + snpFile + - ", target=" + target + "]"); + return; } try { + File target = new File(cacheDir, snpFile.getName()); + + if (log.isDebugEnabled()) { + log.debug("Copying file from the snapshot " + + "[snapshot=" + snpName + + ", src=" + snpFile + + ", target=" + target + "]"); + } + Files.copy(snpFile.toPath(), target.toPath()); } catch (IOException e) { errHnd.accept(e); } + finally { + busyLock.leaveBusy(); + } }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); } } @@ -693,23 +736,35 @@ private IgniteInternalFuture rollback(UUID reqId) { GridFutureAdapter retFut = new GridFutureAdapter<>(); + opCtx0.stopFut = retFut; + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - if (log.isInfoEnabled()) { - log.info("Performing local rollback routine for restored cache groups " + - "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); + if (!busyLock.enterBusy()) { + retFut.onDone(new IgniteCheckedException("Node has been stopped or deactivated.")); + + return; } - for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + try { + if (log.isInfoEnabled()) { + log.info("Performing local rollback routine for restored cache groups " + + "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); + } - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; - U.delete(cacheDir); - } + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); - retFut.onDone(true); + U.delete(cacheDir); + } + + retFut.onDone(true); + } finally { + busyLock.leaveBusy(); + } }); return retFut; @@ -760,6 +815,9 @@ private static class SnapshotRestoreContext { /** Cache ID to configuration mapping. */ private volatile Map cfgs; + /** Graceful shutdown future. */ + private volatile GridFutureAdapter stopFut; + /** * @param req Request to prepare cache group restore from the snapshot. * @param dirs List of cache group names to restore from the snapshot. From c979cc8e5e593d4ce13923118eb2054fdf10d326 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 24 Mar 2021 19:17:32 +0300 Subject: [PATCH 59/98] IGNITE-13805 Graceful shutdown (wip2). --- .../snapshot/SnapshotRestoreProcess.java | 82 +++++-------------- 1 file changed, 22 insertions(+), 60 deletions(-) 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 3a75610471fdf3..014241dd84353b 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 @@ -46,14 +46,12 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; -import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2; import org.apache.ignite.internal.processors.cluster.DiscoveryDataClusterState; -import org.apache.ignite.internal.util.GridBusyLock; import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.future.GridFinishedFuture; import org.apache.ignite.internal.util.future.GridFutureAdapter; @@ -95,9 +93,6 @@ public class SnapshotRestoreProcess { /** Logger. */ private final IgniteLogger log; - /** Busy lock. */ - private final GridBusyLock busyLock = new GridBusyLock(); - /** The future to be completed when the cache restore process is complete. */ private volatile GridFutureAdapter fut; @@ -320,7 +315,7 @@ public void onNodeLeft(UUID leftNodeId) { * * @param reason Interruption reason. */ - public void stop(Exception reason) { + public synchronized void stop(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) @@ -328,18 +323,15 @@ public void stop(Exception reason) { opCtx0.err.compareAndSet(null, reason); - busyLock.block(); - - try { - IgniteInternalFuture stopFut = opCtx0.stopFut; + IgniteInternalFuture stopFut = opCtx0.stopFut; - if (stopFut != null && !stopFut.isDone()) + if (stopFut != null && !stopFut.isDone()) { + try { stopFut.get(); - - } catch (IgniteCheckedException ignore) { - // No-op. - } finally { - busyLock.unblock(); + } + catch (IgniteCheckedException ignore) { + // No-op. + } } } @@ -361,7 +353,7 @@ private void ensureCacheAbsent(String name) { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { + private synchronized IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -390,9 +382,6 @@ private IgniteInternalFuture> prepare(SnapshotRestore SnapshotRestoreContext opCtx0 = opCtx; - if (ctx.isStopping()) - throw new NodeStoppingException("Node is stopping."); - if (opCtx0.dirs.isEmpty()) return new GridFinishedFuture<>(); @@ -473,19 +462,11 @@ private CompletableFuture restoreAsync( File binDir = binaryWorkDir(snapshotMgr.snapshotLocalDir(snpName).getAbsolutePath(), pdsFolderName); futs.add(CompletableFuture.runAsync(() -> { - if (!busyLock.enterBusy()) { - errHnd.accept(new IgniteCheckedException("Node has been stopped or deactivated.")); - - return; - } - try { ctx.cacheObjects().updateMetadata(binDir, stopChecker); } catch (IgniteCheckedException e) { errHnd.accept(e); - } finally { - busyLock.leaveBusy(); } }, snapshotMgr.snapshotExecutorService())); } @@ -501,12 +482,6 @@ private CompletableFuture restoreAsync( if (stopChecker.getAsBoolean()) return; - if (!busyLock.enterBusy()) { - errHnd.accept(new IgniteCheckedException("Node has been stopped or deactivated.")); - - return; - } - try { File target = new File(cacheDir, snpFile.getName()); @@ -522,9 +497,6 @@ private CompletableFuture restoreAsync( catch (IOException e) { errHnd.accept(e); } - finally { - busyLock.leaveBusy(); - } }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); } } @@ -725,7 +697,7 @@ private Exception checkNodeLeft(Set reqNodes, Set respNodes) { * @param reqId Request ID. * @return Result future. */ - private IgniteInternalFuture rollback(UUID reqId) { + private synchronized IgniteInternalFuture rollback(UUID reqId) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -739,32 +711,22 @@ private IgniteInternalFuture rollback(UUID reqId) { opCtx0.stopFut = retFut; ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - if (!busyLock.enterBusy()) { - retFut.onDone(new IgniteCheckedException("Node has been stopped or deactivated.")); - - return; + if (log.isInfoEnabled()) { + log.info("Performing local rollback routine for restored cache groups " + + "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); } - try { - if (log.isInfoEnabled()) { - log.info("Performing local rollback routine for restored cache groups " + - "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); - } - - for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); + if (log.isInfoEnabled()) + log.info("Cleaning up directory " + cacheDir); - U.delete(cacheDir); - } - - retFut.onDone(true); - } finally { - busyLock.leaveBusy(); + U.delete(cacheDir); } + + retFut.onDone(true); }); return retFut; @@ -816,7 +778,7 @@ private static class SnapshotRestoreContext { private volatile Map cfgs; /** Graceful shutdown future. */ - private volatile GridFutureAdapter stopFut; + private GridFutureAdapter stopFut; /** * @param req Request to prepare cache group restore from the snapshot. From dc225b865fc31f3149b2cd5eb3fe6486476def3d Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 25 Mar 2021 13:11:06 +0300 Subject: [PATCH 60/98] IGNITE-13805 Graceful shutdown (wip3). --- .../snapshot/IgniteSnapshotManager.java | 4 +- .../snapshot/SnapshotRestoreProcess.java | 110 ++++++++++++------ 2 files changed, 74 insertions(+), 40 deletions(-) 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 2a8883c6c4f8b7..d6f5047c9f3159 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 @@ -431,7 +431,7 @@ public static String partDeltaFileName(int partId) { busyLock.block(); try { - restoreCacheGrpProc.stop(new NodeStoppingException("Node is stopping.")); + restoreCacheGrpProc.stop(); // Try stop all snapshot processing if not yet. for (SnapshotFutureTask sctx : locSnpTasks.values()) @@ -467,7 +467,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onDeActivate(GridKernalContext kctx) { - restoreCacheGrpProc.stop(new IgniteCheckedException("The cluster has been deactivated.")); + restoreCacheGrpProc.deactivate(); } /** 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 014241dd84353b..25dc4c4674eab8 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 @@ -46,6 +46,7 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; @@ -99,6 +100,9 @@ public class SnapshotRestoreProcess { /** Snapshot restore operation context. */ private volatile SnapshotRestoreContext opCtx; + /** Stopped flag. */ + private volatile boolean stopped; + /** * @param ctx Kernal context. */ @@ -310,12 +314,28 @@ public void onNodeLeft(UUID leftNodeId) { } } + /** + * Abort the currently running restore procedure (if any). + */ + public void stop() { + interrupt(new NodeStoppingException("Node is stopping.")); + + stopped = true; + } + + /** + * Abort the currently running restore procedure (if any). + */ + public void deactivate() { + interrupt(new IgniteCheckedException("The cluster has been deactivated.")); + } + /** * Abort the currently running restore procedure (if any). * * @param reason Interruption reason. */ - public synchronized void stop(Exception reason) { + private synchronized void interrupt(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) @@ -353,7 +373,7 @@ private void ensureCacheAbsent(String name) { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private synchronized IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { + private IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -378,56 +398,62 @@ private synchronized IgniteInternalFuture> prepare(Sn for (String grpName : req.groups()) ensureCacheAbsent(grpName); - opCtx = prepareContext(req); + synchronized (this) { + if (stopped) + throw new NodeStoppingException("Node is stopping."); - SnapshotRestoreContext opCtx0 = opCtx; + opCtx = prepareContext(req); - if (opCtx0.dirs.isEmpty()) - return new GridFinishedFuture<>(); + SnapshotRestoreContext opCtx0 = opCtx; - // Ensure that shared cache groups has no conflicts. - for (StoredCacheData cfg : opCtx0.cfgs.values()) { - if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); - } + if (opCtx0.dirs.isEmpty()) + return new GridFinishedFuture<>(); - if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + - ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); - } + // Ensure that shared cache groups has no conflicts. + for (StoredCacheData cfg : opCtx0.cfgs.values()) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); - Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); - BooleanSupplier stopChecker = () -> { - if (opCtx.err.get() != null) - return true; + if (log.isInfoEnabled()) { + log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + } - if (Thread.currentThread().isInterrupted()) { - errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); + BooleanSupplier stopChecker = () -> { + if (opCtx.err.get() != null) + return true; - return true; - } + if (Thread.currentThread().isInterrupted()) { + errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); - return false; - }; + return true; + } + + return false; + }; - GridFutureAdapter> retFut = new GridFutureAdapter<>(); + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - opCtx0.stopFut = retFut; + opCtx0.stopFut = retFut; - restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { - Throwable err = opCtx.err.get(); + restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { + Throwable err = opCtx.err.get(); - if (err != null) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + if (err != null) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); - retFut.onDone(err); - } else - retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); - }); + retFut.onDone(err); + } + else + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + }); - return retFut; + return retFut; + } } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { log.error("Unable to restore cache group(s) from the snapshot " + "[reqId=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); @@ -706,6 +732,9 @@ private synchronized IgniteInternalFuture rollback(UUID reqId) { if (F.isEmpty(opCtx0.dirs)) return new GridFinishedFuture<>(); + if (stopped) + return new GridFinishedFuture<>(new NodeStoppingException("Node is stopping.")); + GridFutureAdapter retFut = new GridFutureAdapter<>(); opCtx0.stopFut = retFut; @@ -743,6 +772,11 @@ private void finishRollback(UUID reqId, Map res, Map entry : errs.entrySet()) { + log.warning("Remote node was not able to perform rollback " + + "[nodeId=" + entry.getKey() + ", err=" + entry.getValue().getMessage() + ']'); + } + if (!res.keySet().containsAll(opCtx0.nodes)) { Set leftNodes = new HashSet<>(opCtx0.nodes); From c44bffb7bba559aa4aa89594fd505b5edb1f7db4 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 26 Mar 2021 10:13:57 +0300 Subject: [PATCH 61/98] IGNITE-13805 (minor) COde cleanup. --- .../internal/processors/cache/GridCacheProcessor.java | 10 ++++++---- .../persistence/snapshot/SnapshotRestoreProcess.java | 9 ++------- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 8fb59d56834ded..98b49d581f9e26 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3757,6 +3757,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. * @param restartId Restart requester id (it'll allow to start this cache only him). + * @param restored Flag indicating that the cache was started after restoring from a snapshot. * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails * when starting the cache(s), the whole procedure is rolled back. * @return Future that will be completed when all caches are deployed. @@ -3767,7 +3768,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean checkThreadTx, boolean disabledAfterStart, IgniteUuid restartId, - boolean restoredCache, + boolean restored, @Nullable Set topNodes ) { if (checkThreadTx) { @@ -3804,7 +3805,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( ccfg.queryEntities(), ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null, ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null, - restoredCache); + restored); if (req != null) { if (req.clientStartOnly()) { @@ -5088,6 +5089,7 @@ private T withBinaryContext(IgniteOutClosureX c) throws IgniteCheckedExce * @param qryEntities Query entities. * @param encKey Encryption key. * @param masterKeyDigest Master key digest. + * @param restored Flag indicating that the cache was started after restoring from a snapshot. * @return Request or {@code null} if cache already exists. * @throws IgniteCheckedException if some of pre-checks failed * @throws CacheExistsException if cache exists and failIfExists flag is {@code true} @@ -5105,7 +5107,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( @Nullable Collection qryEntities, @Nullable byte[] encKey, @Nullable byte[] masterKeyDigest, - boolean restoredCache + boolean restored ) throws IgniteCheckedException { DynamicCacheDescriptor desc = cacheDescriptor(cacheName); @@ -5123,7 +5125,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( req.restartId(restartId); - req.restoredCache(restoredCache); + req.restoredCache(restored); if (ccfg != null) { cloneCheckSerializable(ccfg); 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 25dc4c4674eab8..260b3cf410b6a4 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 @@ -649,11 +649,6 @@ private IgniteInternalFuture cacheStart(UUID reqId) { SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0 == null) { - return new GridFinishedFuture<>(new IgniteIllegalStateException("Context has not been created on server " + - "node during prepare operation [reqId=" + reqId + ", nodeId=" + ctx.localNodeId() + ']')); - } - Throwable err = opCtx0.err.get(); if (err != null) @@ -770,13 +765,13 @@ private void finishRollback(UUID reqId, Map res, Map entry : errs.entrySet()) { log.warning("Remote node was not able to perform rollback " + "[nodeId=" + entry.getKey() + ", err=" + entry.getValue().getMessage() + ']'); } + SnapshotRestoreContext opCtx0 = opCtx; + if (!res.keySet().containsAll(opCtx0.nodes)) { Set leftNodes = new HashSet<>(opCtx0.nodes); From df3032e83862c977a686f2ac60a853263187c10b Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 26 Mar 2021 10:45:37 +0300 Subject: [PATCH 62/98] IGNITE-13805 No need to check stopping flag in prepare. --- .../snapshot/SnapshotRestoreProcess.java | 80 +++++++++---------- 1 file changed, 39 insertions(+), 41 deletions(-) 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 260b3cf410b6a4..38613d074b5896 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 @@ -398,62 +398,60 @@ private IgniteInternalFuture> prepare(SnapshotRestore for (String grpName : req.groups()) ensureCacheAbsent(grpName); - synchronized (this) { - if (stopped) - throw new NodeStoppingException("Node is stopping."); + if (ctx.isStopping()) + throw new NodeStoppingException("Node is stopping."); - opCtx = prepareContext(req); + opCtx = prepareContext(req); - SnapshotRestoreContext opCtx0 = opCtx; + SnapshotRestoreContext opCtx0 = opCtx; - if (opCtx0.dirs.isEmpty()) - return new GridFinishedFuture<>(); + if (opCtx0.dirs.isEmpty()) + return new GridFinishedFuture<>(); - // Ensure that shared cache groups has no conflicts. - for (StoredCacheData cfg : opCtx0.cfgs.values()) { - if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); - } + // Ensure that shared cache groups has no conflicts. + for (StoredCacheData cfg : opCtx0.cfgs.values()) { + if (!F.isEmpty(cfg.config().getGroupName())) + ensureCacheAbsent(cfg.config().getName()); + } - if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + - ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); - } + if (log.isInfoEnabled()) { + log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + + ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + } - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); - Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); - BooleanSupplier stopChecker = () -> { - if (opCtx.err.get() != null) - return true; + boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); + Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); + BooleanSupplier stopChecker = () -> { + if (opCtx.err.get() != null) + return true; - if (Thread.currentThread().isInterrupted()) { - errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); + if (Thread.currentThread().isInterrupted()) { + errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); - return true; - } + return true; + } - return false; - }; + return false; + }; - GridFutureAdapter> retFut = new GridFutureAdapter<>(); + GridFutureAdapter> retFut = new GridFutureAdapter<>(); - opCtx0.stopFut = retFut; + opCtx0.stopFut = retFut; - restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { - Throwable err = opCtx.err.get(); + restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { + Throwable err = opCtx.err.get(); - if (err != null) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + if (err != null) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); - retFut.onDone(err); - } - else - retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); - }); + retFut.onDone(err); + } + else + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + }); - return retFut; - } + return retFut; } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { log.error("Unable to restore cache group(s) from the snapshot " + "[reqId=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); From a13341d2b929b489a613c5b8f977fa2bcada795b Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 29 Mar 2021 00:16:31 +0300 Subject: [PATCH 63/98] IGNITE-13805 (minor) restored -> internal --- .../processors/cache/ClusterCachesInfo.java | 2 +- .../cache/DynamicCacheChangeRequest.java | 16 ++++++++-------- .../processors/cache/GridCacheProcessor.java | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 853b5b9bb38a41..40a7515fd574f4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -1028,7 +1028,7 @@ else if (encMgr.masterKeyDigest() != null && } } - if (err == null && !req.restoredCache()) { + if (err == null && !req.internal()) { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); if (snapshotMgr.isRestoring(cacheName, ccfg.getGroupName())) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java index 6293fb1dcba2b7..a97ca08d3b97f7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java @@ -107,8 +107,8 @@ public class DynamicCacheChangeRequest implements Serializable { /** Cache configuration enrichment. */ private CacheConfigurationEnrichment cacheCfgEnrichment; - /** Flag indicating that the cache was started after restoring from a snapshot. */ - private boolean restoredCache; + /** Flag indicating that the cache was started internally and not by the user. */ + private boolean internal; /** * @param reqId Unique request ID. @@ -484,17 +484,17 @@ public void masterKeyDigest(@Nullable byte[] masterKeyDigest) { } /** - * @param restoredCache Flag indicating that the cache was started after restoring from a snapshot. + * @param internal Flag indicating that the cache was started internally and not by the user. */ - public void restoredCache(boolean restoredCache) { - this.restoredCache = restoredCache; + public void internal(boolean internal) { + this.internal = internal; } /** - * @return Flag indicating that the cache was started after restoring from a snapshot. + * @return Flag indicating that the cache was started internally and not by the user. */ - public boolean restoredCache() { - return restoredCache; + public boolean internal() { + return internal; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 98b49d581f9e26..a9678b19c0f92e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3757,7 +3757,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. * @param restartId Restart requester id (it'll allow to start this cache only him). - * @param restored Flag indicating that the cache was started after restoring from a snapshot. + * @param internal Flag indicating that the cache was started internally and not by the user. * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails * when starting the cache(s), the whole procedure is rolled back. * @return Future that will be completed when all caches are deployed. @@ -3768,7 +3768,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean checkThreadTx, boolean disabledAfterStart, IgniteUuid restartId, - boolean restored, + boolean internal, @Nullable Set topNodes ) { if (checkThreadTx) { @@ -3805,7 +3805,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( ccfg.queryEntities(), ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null, ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null, - restored); + internal); if (req != null) { if (req.clientStartOnly()) { @@ -5089,7 +5089,7 @@ private T withBinaryContext(IgniteOutClosureX c) throws IgniteCheckedExce * @param qryEntities Query entities. * @param encKey Encryption key. * @param masterKeyDigest Master key digest. - * @param restored Flag indicating that the cache was started after restoring from a snapshot. + * @param internal Flag indicating that the cache was started internally and not by the user. * @return Request or {@code null} if cache already exists. * @throws IgniteCheckedException if some of pre-checks failed * @throws CacheExistsException if cache exists and failIfExists flag is {@code true} @@ -5107,7 +5107,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( @Nullable Collection qryEntities, @Nullable byte[] encKey, @Nullable byte[] masterKeyDigest, - boolean restored + boolean internal ) throws IgniteCheckedException { DynamicCacheDescriptor desc = cacheDescriptor(cacheName); @@ -5125,7 +5125,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( req.restartId(restartId); - req.restoredCache(restored); + req.internal(internal); if (ccfg != null) { cloneCheckSerializable(ccfg); From 50cd43478208dc7d4c672bf646be9a2a5bc1c155 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 11:07:14 +0300 Subject: [PATCH 64/98] IGNITE-13805 Improve stop sync. --- .../snapshot/SnapshotRestoreProcess.java | 57 +++++++++++------- .../IgniteClusterSnapshotRestoreSelfTest.java | 60 +------------------ 2 files changed, 35 insertions(+), 82 deletions(-) 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 38613d074b5896..b6987fc8d3bd0b 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 @@ -318,24 +318,23 @@ public void onNodeLeft(UUID leftNodeId) { * Abort the currently running restore procedure (if any). */ public void stop() { - interrupt(new NodeStoppingException("Node is stopping.")); - - stopped = true; + interrupt(new NodeStoppingException("Node is stopping."), true); } /** * Abort the currently running restore procedure (if any). */ public void deactivate() { - interrupt(new IgniteCheckedException("The cluster has been deactivated.")); + interrupt(new IgniteCheckedException("The cluster has been deactivated."), false); } /** * Abort the currently running restore procedure (if any). * * @param reason Interruption reason. + * @param stop Stop flag. */ - private synchronized void interrupt(Exception reason) { + private void interrupt(Exception reason, boolean stop) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) @@ -343,15 +342,23 @@ private synchronized void interrupt(Exception reason) { opCtx0.err.compareAndSet(null, reason); - IgniteInternalFuture stopFut = opCtx0.stopFut; + IgniteInternalFuture stopFut; - if (stopFut != null && !stopFut.isDone()) { - try { - stopFut.get(); - } - catch (IgniteCheckedException ignore) { - // No-op. - } + synchronized (this) { + stopFut = opCtx0.stopFut; + + if (stop) + stopped = true; + } + + if (stopFut == null || stopFut.isDone()) + return; + + try { + stopFut.get(); + } + catch (IgniteCheckedException ignore) { + // No-op. } } @@ -398,9 +405,6 @@ private IgniteInternalFuture> prepare(SnapshotRestore for (String grpName : req.groups()) ensureCacheAbsent(grpName); - if (ctx.isStopping()) - throw new NodeStoppingException("Node is stopping."); - opCtx = prepareContext(req); SnapshotRestoreContext opCtx0 = opCtx; @@ -436,7 +440,12 @@ private IgniteInternalFuture> prepare(SnapshotRestore GridFutureAdapter> retFut = new GridFutureAdapter<>(); - opCtx0.stopFut = retFut; + synchronized (this) { + if (stopped || ctx.isStopping()) + throw new NodeStoppingException("Node is stopping."); + + opCtx0.stopFut = retFut.chain(f -> null); + } restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { Throwable err = opCtx.err.get(); @@ -716,7 +725,7 @@ private Exception checkNodeLeft(Set reqNodes, Set respNodes) { * @param reqId Request ID. * @return Result future. */ - private synchronized IgniteInternalFuture rollback(UUID reqId) { + private IgniteInternalFuture rollback(UUID reqId) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -725,12 +734,14 @@ private synchronized IgniteInternalFuture rollback(UUID reqId) { if (F.isEmpty(opCtx0.dirs)) return new GridFinishedFuture<>(); - if (stopped) - return new GridFinishedFuture<>(new NodeStoppingException("Node is stopping.")); - GridFutureAdapter retFut = new GridFutureAdapter<>(); - opCtx0.stopFut = retFut; + synchronized (this) { + if (stopped) + return new GridFinishedFuture<>(new NodeStoppingException("Node is stopping.")); + + opCtx0.stopFut = retFut.chain(f -> null); + } ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { if (log.isInfoEnabled()) { @@ -805,7 +816,7 @@ private static class SnapshotRestoreContext { private volatile Map cfgs; /** Graceful shutdown future. */ - private GridFutureAdapter stopFut; + private IgniteInternalFuture stopFut; /** * @param req Request to prepare cache group restore from the snapshot. 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 3f0f4a6835b4c9..33aac59c5a252c 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 @@ -34,7 +34,6 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cache.CacheExistsException; -import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.annotations.QuerySqlField; @@ -101,7 +100,7 @@ else if (dfltCacheCfg != null) { */ private QueryEntity queryEntity(String typeName) { return new QueryEntity() - .setKeyType("java.lang.Integer") + .setKeyType(Integer.class.getName()) .setValueType(typeName) .setFields(new LinkedHashMap<>(F.asMap("id", Integer.class.getName(), "name", String.class.getName()))) .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); @@ -285,63 +284,6 @@ public void testRestoreSharedCacheGroup() throws Exception { checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); } - /** @throws Exception If failed. */ - @Test - public void testRestoreCacheGroupWithNodeFilter() throws Exception { - String cacheName1 = "cache1"; - String cacheName2 = "cache2"; - - CacheConfiguration cacheCfg1 = - txCacheConfig(new CacheConfiguration(cacheName1)).setCacheMode(CacheMode.REPLICATED); - - CacheConfiguration cacheCfg2 = - txCacheConfig(new CacheConfiguration(cacheName2)).setCacheMode(CacheMode.REPLICATED); - - IgniteEx ignite0 = startGrid(0); - IgniteEx ignite1 = startGrid(1); - - ignite0.cluster().state(ClusterState.ACTIVE); - - UUID nodeId0 = ignite0.localNode().id(); - UUID nodeId1 = ignite1.localNode().id(); - - cacheCfg1.setNodeFilter(node -> node.id().equals(nodeId0)); - cacheCfg2.setNodeFilter(node -> node.id().equals(nodeId1)); - - putKeys(ignite0.cache(dfltCacheCfg.getName()), 0, CACHE_KEYS_RANGE); - - IgniteCache cache1 = ignite0.createCache(cacheCfg1); - putKeys(cache1, 0, CACHE_KEYS_RANGE); - - IgniteCache cache2 = ignite0.createCache(cacheCfg2); - putKeys(cache2, 0, CACHE_KEYS_RANGE); - - ignite0.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - - cache1.destroy(); - cache2.destroy(); - - forceCheckpoint(); - - awaitPartitionMapExchange(); - - // After destroying the cache with a node filter, the configuration file remains on the filtered node. - // todo https://issues.apache.org/jira/browse/IGNITE-14044 - for (String cacheName : new String[] {cacheName1, cacheName2}) { - for (int nodeIdx = 0; nodeIdx < 2; nodeIdx++) - U.delete(resolveCacheDir(grid(nodeIdx), cacheName)); - } - - ignite1.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName2)).get(TIMEOUT); - awaitPartitionMapExchange(); - - ignite0.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName1)).get(TIMEOUT); - awaitPartitionMapExchange(); - - checkCacheKeys(ignite0.cache(cacheName1), CACHE_KEYS_RANGE); - checkCacheKeys(ignite0.cache(cacheName2), CACHE_KEYS_RANGE); - } - /** @throws Exception If failed. */ @Test public void testIncompatibleMetasUpdate() throws Exception { From fd9b9b76aa9232b6f09cbb9a2bcd639f53719328 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 18:09:43 +0300 Subject: [PATCH 65/98] IGNITE-13805 Review notes. --- .../CacheObjectBinaryProcessorImpl.java | 4 + .../snapshot/SnapshotRestoreProcess.java | 156 +++++++++--------- 2 files changed, 80 insertions(+), 80 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index 1d2e6ddfc5f36d..a7e1bce7d77989 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -56,6 +56,7 @@ import org.apache.ignite.internal.IgniteFeatures; import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.UnregisteredBinaryTypeException; @@ -1019,6 +1020,9 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { if (stopChecker.getAsBoolean()) return; + if (Thread.currentThread().isInterrupted()) + throw new IgniteInterruptedCheckedException("Thread has been interrupted."); + addMeta(newMeta.typeId(), newMeta.wrap(binaryContext()), false); } } catch (BinaryObjectException e) { 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 b6987fc8d3bd0b..c14d47a11de7be 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 @@ -1,19 +1,19 @@ -///* -// * 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. -// */ +/* + * 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; @@ -94,7 +94,7 @@ public class SnapshotRestoreProcess { /** Logger. */ private final IgniteLogger log; - /** The future to be completed when the cache restore process is complete. */ + /** Future to be completed when the cache restore process is complete (this future will be returned to the user). */ private volatile GridFutureAdapter fut; /** Snapshot restore operation context. */ @@ -148,7 +148,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); synchronized (this) { - if (isRestoring()) + if (isRestoring() && fut == null) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); fut = new GridFutureAdapter<>(); @@ -227,7 +227,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames * @return {@code True} if the snapshot restore operation is in progress. */ public boolean isRestoring() { - return opCtx != null || fut != null; + return isRestoring(null, null); } /** @@ -237,12 +237,15 @@ public boolean isRestoring() { * @param grpName Cache group name. * @return {@code True} if the cache or group with the specified name is currently being restored. */ - public boolean isRestoring(String cacheName, @Nullable String grpName) { + public boolean isRestoring(@Nullable String cacheName, @Nullable String grpName) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) return false; + if (cacheName == null) + return true; + Map cacheCfgs = opCtx0.cfgs; int cacheId = CU.cacheId(cacheName); @@ -291,12 +294,14 @@ else if (log.isInfoEnabled()) opCtx = null; - GridFutureAdapter fut0 = fut; + synchronized (this) { + GridFutureAdapter fut0 = fut; - if (fut0 != null) { - fut = null; + if (fut0 != null) { + fut = null; - ctx.getSystemExecutorService().submit(() -> fut0.onDone(null, err)); + ctx.getSystemExecutorService().submit(() -> fut0.onDone(null, err)); + } } } @@ -342,7 +347,7 @@ private void interrupt(Exception reason, boolean stop) { opCtx0.err.compareAndSet(null, reason); - IgniteInternalFuture stopFut; + IgniteFuture stopFut; synchronized (this) { stopFut = opCtx0.stopFut; @@ -351,15 +356,8 @@ private void interrupt(Exception reason, boolean stop) { stopped = true; } - if (stopFut == null || stopFut.isDone()) - return; - - try { + if (stopFut != null) stopFut.get(); - } - catch (IgniteCheckedException ignore) { - // No-op. - } } /** @@ -423,42 +421,30 @@ private IgniteInternalFuture> prepare(SnapshotRestore ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); } - boolean updateMeta = ctx.localNodeId().equals(req.updateMetaNodeId()); - Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); - BooleanSupplier stopChecker = () -> { - if (opCtx.err.get() != null) - return true; - - if (Thread.currentThread().isInterrupted()) { - errHnd.accept(new IgniteInterruptedCheckedException("Thread has been interrupted.")); - - return true; - } - - return false; - }; - + Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); + BooleanSupplier stopChecker = () -> opCtx.err.get() != null; GridFutureAdapter> retFut = new GridFutureAdapter<>(); synchronized (this) { if (stopped || ctx.isStopping()) throw new NodeStoppingException("Node is stopping."); - opCtx0.stopFut = retFut.chain(f -> null); + opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); } - restoreAsync(opCtx0.snpName, opCtx0.dirs, updateMeta, stopChecker, errHnd).thenAccept(res -> { - Throwable err = opCtx.err.get(); + restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.updateMetaNodeId()), stopChecker, errHnd) + .thenAccept(res -> { + Throwable err = opCtx.err.get(); - if (err != null) { - log.error("Unable to restore cache group(s) from the snapshot " + - "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + if (err != null) { + log.error("Unable to restore cache group(s) from the snapshot " + + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); - retFut.onDone(err); - } - else - retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); - }); + retFut.onDone(err); + } + else + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + }); return retFut; } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { @@ -484,7 +470,7 @@ private CompletableFuture restoreAsync( Collection dirs, boolean updateMeta, BooleanSupplier stopChecker, - Consumer errHnd + Consumer errHnd ) throws IgniteCheckedException { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); String pdsFolderName = ctx.pdsFolderResolver().resolveFolders().folderName(); @@ -498,8 +484,8 @@ private CompletableFuture restoreAsync( try { ctx.cacheObjects().updateMetadata(binDir, stopChecker); } - catch (IgniteCheckedException e) { - errHnd.accept(e); + catch (Throwable t) { + errHnd.accept(t); } }, snapshotMgr.snapshotExecutorService())); } @@ -516,6 +502,9 @@ private CompletableFuture restoreAsync( return; try { + if (Thread.currentThread().isInterrupted()) + throw new IgniteInterruptedCheckedException("Thread has been interrupted."); + File target = new File(cacheDir, snpFile.getName()); if (log.isDebugEnabled()) { @@ -527,7 +516,7 @@ private CompletableFuture restoreAsync( Files.copy(snpFile.toPath(), target.toPath()); } - catch (IOException e) { + catch (IgniteInterruptedCheckedException | IOException e) { errHnd.accept(e); } }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); @@ -740,27 +729,34 @@ private IgniteInternalFuture rollback(UUID reqId) { if (stopped) return new GridFinishedFuture<>(new NodeStoppingException("Node is stopping.")); - opCtx0.stopFut = retFut.chain(f -> null); + opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); } - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - if (log.isInfoEnabled()) { - log.info("Performing local rollback routine for restored cache groups " + - "[reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'); - } + try { + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + if (log.isInfoEnabled()) { + log.info("Removing restored cache directories [reqId=" + opCtx0.reqId + + ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); + } - for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + IgniteCheckedException ex = null; - if (log.isInfoEnabled()) - log.info("Cleaning up directory " + cacheDir); + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; - U.delete(cacheDir); - } + if (!U.delete(cacheDir)) + ex = new IgniteCheckedException("Unable to remove directory " + cacheDir); + } - retFut.onDone(true); - }); + if (ex != null) + retFut.onDone(ex); + else + retFut.onDone(true); + }); + } catch (RejectedExecutionException e) { + retFut.onDone(e); + } return retFut; } @@ -810,13 +806,13 @@ private static class SnapshotRestoreContext { private final Collection dirs; /** The exception that led to the interruption of the process. */ - private final AtomicReference err = new AtomicReference<>(); + private final AtomicReference err = new AtomicReference<>(); /** Cache ID to configuration mapping. */ private volatile Map cfgs; /** Graceful shutdown future. */ - private IgniteInternalFuture stopFut; + private IgniteFuture stopFut; /** * @param req Request to prepare cache group restore from the snapshot. From 50165a34e7ed4b5ef5501b1c0f9f9924d6eff208 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 18:45:42 +0300 Subject: [PATCH 66/98] IGNITE-13805 Remove stopped flag. --- .../snapshot/SnapshotRestoreProcess.java | 27 ++++--------------- 1 file changed, 5 insertions(+), 22 deletions(-) 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 c14d47a11de7be..57e2b250d873aa 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 @@ -100,9 +100,6 @@ public class SnapshotRestoreProcess { /** Snapshot restore operation context. */ private volatile SnapshotRestoreContext opCtx; - /** Stopped flag. */ - private volatile boolean stopped; - /** * @param ctx Kernal context. */ @@ -347,14 +344,7 @@ private void interrupt(Exception reason, boolean stop) { opCtx0.err.compareAndSet(null, reason); - IgniteFuture stopFut; - - synchronized (this) { - stopFut = opCtx0.stopFut; - - if (stop) - stopped = true; - } + IgniteFuture stopFut = opCtx0.stopFut; if (stopFut != null) stopFut.get(); @@ -425,12 +415,10 @@ private IgniteInternalFuture> prepare(SnapshotRestore BooleanSupplier stopChecker = () -> opCtx.err.get() != null; GridFutureAdapter> retFut = new GridFutureAdapter<>(); - synchronized (this) { - if (stopped || ctx.isStopping()) - throw new NodeStoppingException("Node is stopping."); + if (ctx.isStopping()) + throw new NodeStoppingException("Node is stopping."); - opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); - } + opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.updateMetaNodeId()), stopChecker, errHnd) .thenAccept(res -> { @@ -725,12 +713,7 @@ private IgniteInternalFuture rollback(UUID reqId) { GridFutureAdapter retFut = new GridFutureAdapter<>(); - synchronized (this) { - if (stopped) - return new GridFinishedFuture<>(new NodeStoppingException("Node is stopping.")); - - opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); - } + opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); try { ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { From db846c56a656ed8399aecd9a6e575787f3ffb4b8 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 18:51:30 +0300 Subject: [PATCH 67/98] IGNITE-13805 Review notes. --- .../snapshot/SnapshotRestoreProcess.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) 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 57e2b250d873aa..bada2af4098e10 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 @@ -157,7 +157,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - finishProcess(f.error()); + fut.onDone(f.error()); + + fut = null; return; } @@ -180,9 +182,11 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - finishProcess(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); + fut = null; + return; } @@ -201,7 +205,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - finishProcess(new IgniteException(sb.toString())); + fut.onDone(new IgniteException(sb.toString())); + + fut = null; return; } @@ -320,23 +326,22 @@ public void onNodeLeft(UUID leftNodeId) { * Abort the currently running restore procedure (if any). */ public void stop() { - interrupt(new NodeStoppingException("Node is stopping."), true); + interrupt(new NodeStoppingException("Node is stopping.")); } /** * Abort the currently running restore procedure (if any). */ public void deactivate() { - interrupt(new IgniteCheckedException("The cluster has been deactivated."), false); + interrupt(new IgniteCheckedException("The cluster has been deactivated.")); } /** * Abort the currently running restore procedure (if any). * * @param reason Interruption reason. - * @param stop Stop flag. */ - private void interrupt(Exception reason, boolean stop) { + private void interrupt(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) @@ -795,7 +800,7 @@ private static class SnapshotRestoreContext { private volatile Map cfgs; /** Graceful shutdown future. */ - private IgniteFuture stopFut; + private volatile IgniteFuture stopFut; /** * @param req Request to prepare cache group restore from the snapshot. From ee9888c5b0223a66acd1cd7f81ede3703c8c0dec Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 19:08:24 +0300 Subject: [PATCH 68/98] IGNITE-13805 Added sync for rollback. --- .../snapshot/SnapshotRestoreProcess.java | 51 +++++++++++-------- 1 file changed, 29 insertions(+), 22 deletions(-) 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 bada2af4098e10..5832ab2fa0d4c9 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 @@ -349,7 +349,11 @@ private void interrupt(Exception reason) { opCtx0.err.compareAndSet(null, reason); - IgniteFuture stopFut = opCtx0.stopFut; + IgniteFuture stopFut; + + synchronized (this) { + stopFut = opCtx0.stopFut; + } if (stopFut != null) stopFut.get(); @@ -718,32 +722,35 @@ private IgniteInternalFuture rollback(UUID reqId) { GridFutureAdapter retFut = new GridFutureAdapter<>(); - opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); + synchronized (this) { + opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); - try { - ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { - if (log.isInfoEnabled()) { - log.info("Removing restored cache directories [reqId=" + opCtx0.reqId + - ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); - } + try { + ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { + if (log.isInfoEnabled()) { + log.info("Removing restored cache directories [reqId=" + opCtx0.reqId + + ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); + } - IgniteCheckedException ex = null; + IgniteCheckedException ex = null; - for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + for (File cacheDir : opCtx0.dirs) { + if (!cacheDir.exists()) + continue; - if (!U.delete(cacheDir)) - ex = new IgniteCheckedException("Unable to remove directory " + cacheDir); - } + if (!U.delete(cacheDir)) + ex = new IgniteCheckedException("Unable to remove directory " + cacheDir); + } - if (ex != null) - retFut.onDone(ex); - else - retFut.onDone(true); - }); - } catch (RejectedExecutionException e) { - retFut.onDone(e); + if (ex != null) + retFut.onDone(ex); + else + retFut.onDone(true); + }); + } + catch (RejectedExecutionException e) { + retFut.onDone(e); + } } return retFut; From 1cf1c297daa8232184e9464332bcfaa695d6e798 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 19:49:10 +0300 Subject: [PATCH 69/98] IGNITE-13805 Remove stop/deactivater meths. --- .../snapshot/IgniteSnapshotManager.java | 6 +++--- .../snapshot/SnapshotRestoreProcess.java | 16 +--------------- 2 files changed, 4 insertions(+), 18 deletions(-) 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 d6f5047c9f3159..79af53559735e7 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 @@ -431,7 +431,7 @@ public static String partDeltaFileName(int partId) { busyLock.block(); try { - restoreCacheGrpProc.stop(); + restoreCacheGrpProc.interrupt(new NodeStoppingException("Node is stopping.")); // Try stop all snapshot processing if not yet. for (SnapshotFutureTask sctx : locSnpTasks.values()) @@ -467,7 +467,7 @@ public static String partDeltaFileName(int partId) { /** {@inheritDoc} */ @Override public void onDeActivate(GridKernalContext kctx) { - restoreCacheGrpProc.deactivate(); + restoreCacheGrpProc.interrupt(new IgniteCheckedException("The cluster has been deactivated.")); } /** @@ -1370,7 +1370,7 @@ private void recordSnapshotEvent(String snpName, String msg, int type) { /** * @return The executor used to run snapshot tasks. */ - Executor snapshotExecutorService() { + ExecutorService snapshotExecutorService() { assert snpRunner != null; return snpRunner; 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 5832ab2fa0d4c9..3de209a2c784d1 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 @@ -322,26 +322,12 @@ public void onNodeLeft(UUID leftNodeId) { } } - /** - * Abort the currently running restore procedure (if any). - */ - public void stop() { - interrupt(new NodeStoppingException("Node is stopping.")); - } - - /** - * Abort the currently running restore procedure (if any). - */ - public void deactivate() { - interrupt(new IgniteCheckedException("The cluster has been deactivated.")); - } - /** * Abort the currently running restore procedure (if any). * * @param reason Interruption reason. */ - private void interrupt(Exception reason) { + public void interrupt(Exception reason) { SnapshotRestoreContext opCtx0 = opCtx; if (opCtx0 == null) From 135e3596a281b650d1eb99ad88297ebc42c6b1be Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 19:51:34 +0300 Subject: [PATCH 70/98] IGNITE-13805 Bugfix. --- .../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 3de209a2c784d1..64946ffd8c9619 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 @@ -145,7 +145,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames throw new IgniteException(OP_REJECT_MSG + "A cluster snapshot operation is in progress."); synchronized (this) { - if (isRestoring() && fut == null) + if (isRestoring() || fut != null) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); fut = new GridFutureAdapter<>(); From b6b12c17f39c36409a8b5613cfe6958a6d8b689e Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 20:20:37 +0300 Subject: [PATCH 71/98] IGNITE-13805 Rollback logging improvement --- .../CacheObjectBinaryProcessorImpl.java | 2 +- .../snapshot/SnapshotRestoreProcess.java | 21 ++++++++++++------- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java index a7e1bce7d77989..15619d398bb37d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java @@ -1020,7 +1020,7 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException { if (stopChecker.getAsBoolean()) return; - if (Thread.currentThread().isInterrupted()) + if (Thread.interrupted()) throw new IgniteInterruptedCheckedException("Thread has been interrupted."); addMeta(newMeta.typeId(), newMeta.wrap(binaryContext()), false); 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 64946ffd8c9619..3204cd243ab64e 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 @@ -485,7 +485,7 @@ private CompletableFuture restoreAsync( return; try { - if (Thread.currentThread().isInterrupted()) + if (Thread.interrupted()) throw new IgniteInterruptedCheckedException("Thread has been interrupted."); File target = new File(cacheDir, snpFile.getName()); @@ -714,7 +714,7 @@ private IgniteInternalFuture rollback(UUID reqId) { try { ctx.cache().context().snapshotMgr().snapshotExecutorService().execute(() -> { if (log.isInfoEnabled()) { - log.info("Removing restored cache directories [reqId=" + opCtx0.reqId + + log.info("Removing restored cache directories [reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dirs=" + opCtx0.dirs + ']'); } @@ -724,8 +724,12 @@ private IgniteInternalFuture rollback(UUID reqId) { if (!cacheDir.exists()) continue; - if (!U.delete(cacheDir)) - ex = new IgniteCheckedException("Unable to remove directory " + cacheDir); + if (!U.delete(cacheDir)) { + log.error("Unable to perform rollback routine completely, cannot remove cache directory " + + "[reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dir=" + cacheDir + ']'); + + ex = new IgniteCheckedException("Unable to remove cache directory " + cacheDir); + } } if (ex != null) @@ -735,6 +739,9 @@ private IgniteInternalFuture rollback(UUID reqId) { }); } catch (RejectedExecutionException e) { + log.error("Unable to perform rollback routine, task has been rejected " + + "[reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ']'); + retFut.onDone(e); } } @@ -751,9 +758,9 @@ private void finishRollback(UUID reqId, Map res, Map entry : errs.entrySet()) { - log.warning("Remote node was not able to perform rollback " + - "[nodeId=" + entry.getKey() + ", err=" + entry.getValue().getMessage() + ']'); + if (!errs.isEmpty()) { + log.warning("Some nodes were unable to complete the rollback routine completely, check the local log " + + "files for more information [nodeIds=" + errs.keySet() + ']'); } SnapshotRestoreContext opCtx0 = opCtx; From a446660fd61febed9fe934c7221abb12c045ae97 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 5 Apr 2021 20:49:54 +0300 Subject: [PATCH 72/98] IGNITE-13805 Code cleanup. --- .../snapshot/SnapshotRestoreProcess.java | 48 +++++++++---------- 1 file changed, 23 insertions(+), 25 deletions(-) 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 3204cd243ab64e..cf0e2cb6a474f9 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 @@ -154,12 +154,12 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return new IgniteFinishedFutureImpl<>(e); } + UUID reqId = UUID.randomUUID(); + ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - fut.onDone(f.error()); - - fut = null; + finishProcess(reqId, f.error()); return; } @@ -182,18 +182,16 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - fut.onDone(new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + finishProcess(reqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); - fut = null; - return; } ctx.cache().context().snapshotMgr().runSnapshotVerfification(metas).listen( f0 -> { if (f0.error() != null) { - fut.onDone(f0.error()); + finishProcess(reqId, f0.error()); return; } @@ -205,15 +203,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - fut.onDone(new IgniteException(sb.toString())); - - fut = null; + finishProcess(reqId, new IgniteException(sb.toString())); return; } - SnapshotRestoreRequest req = new SnapshotRestoreRequest(UUID.randomUUID(), - snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); + SnapshotRestoreRequest req = + new SnapshotRestoreRequest(reqId, snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); prepareRestoreProc.start(req.requestId(), req); } @@ -275,27 +271,29 @@ else if (CU.cacheId(locGrpName) == cacheId) /** * Finish local cache group restore process. + * + * @param reqId Request ID. */ - private void finishProcess() { - finishProcess(null); + private void finishProcess(UUID reqId) { + finishProcess(reqId, null); } /** * Finish local cache group restore process. * + * @param reqId Request ID. * @param err Error, if any. */ - private void finishProcess(@Nullable Throwable err) { - SnapshotRestoreContext opCtx0 = opCtx; - - String details = opCtx0 == null ? "" : " [reqId=" + opCtx0.reqId + ", snapshot=" + opCtx0.snpName + ']'; - + private void finishProcess(UUID reqId, @Nullable Throwable err) { if (err != null) - log.error("Failed to restore snapshot cache group" + details, err); + log.error("Failed to restore snapshot cache group [reqId=" + reqId + ']', err); else if (log.isInfoEnabled()) - log.info("Successfully restored cache group(s) from the snapshot" + details); + log.info("Successfully restored cache group(s) from the snapshot [reqId=" + reqId + ']'); + + SnapshotRestoreContext opCtx0 = opCtx; - opCtx = null; + if (opCtx0 != null && reqId.equals(opCtx0.reqId)) + opCtx = null; synchronized (this) { GridFutureAdapter fut0 = fut; @@ -584,7 +582,7 @@ private void finishPrepare(UUID reqId, Map> res assert opCtx0 != null || failure != null : ctx.localNodeId(); if (opCtx0 == null) { - finishProcess(failure); + finishProcess(reqId, failure); return; } @@ -664,7 +662,7 @@ private void finishCacheStart(UUID reqId, Map res, Map res, Map(); + fut = new ClusterSnapshotFuture(UUID.randomUUID(), snpName); } } catch (IgniteException e) { return new IgniteFinishedFutureImpl<>(e); } - UUID reqId = UUID.randomUUID(); - ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - finishProcess(reqId, f.error()); + finishProcess(fut.rqId, f.error()); return; } @@ -182,7 +181,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - finishProcess(reqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + finishProcess(fut.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); return; @@ -191,7 +190,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().runSnapshotVerfification(metas).listen( f0 -> { if (f0.error() != null) { - finishProcess(reqId, f0.error()); + finishProcess(fut.rqId, f0.error()); return; } @@ -203,13 +202,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - finishProcess(reqId, new IgniteException(sb.toString())); + finishProcess(fut.rqId, new IgniteException(sb.toString())); return; } SnapshotRestoreRequest req = - new SnapshotRestoreRequest(reqId, snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); + new SnapshotRestoreRequest(fut.rqId, snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); prepareRestoreProc.start(req.requestId(), req); } @@ -296,9 +295,9 @@ else if (log.isInfoEnabled()) opCtx = null; synchronized (this) { - GridFutureAdapter fut0 = fut; + ClusterSnapshotFuture fut0 = fut; - if (fut0 != null) { + if (fut0 != null && reqId.equals(fut0.rqId)) { fut = null; ctx.getSystemExecutorService().submit(() -> fut0.onDone(null, err)); From 2807e84afd7b1219ad5daa9fcbd571fe4e110e48 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 6 Apr 2021 13:21:29 +0300 Subject: [PATCH 74/98] IGNITE-13805 Duplicated operation request. --- .../snapshot/IgniteSnapshotManager.java | 108 ++++++------------ ...est.java => SnapshotOperationRequest.java} | 55 ++++++--- .../snapshot/SnapshotRestoreProcess.java | 14 +-- 3 files changed, 78 insertions(+), 99 deletions(-) rename modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/{SnapshotRestoreRequest.java => SnapshotOperationRequest.java} (64%) 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 0e513c65e04548..5f0f49561efbbb 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 @@ -51,7 +51,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.Executor; import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.function.BiFunction; @@ -122,7 +121,6 @@ import org.apache.ignite.internal.util.lang.GridPlainRunnable; import org.apache.ignite.internal.util.lang.IgniteThrowableFunction; import org.apache.ignite.internal.util.tostring.GridToStringExclude; -import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.A; import org.apache.ignite.internal.util.typedef.internal.CU; @@ -410,8 +408,8 @@ public static String partDeltaFileName(int partId) { for (SnapshotFutureTask sctx : locSnpTasks.values()) { if (sctx.sourceNodeId().equals(leftNodeId) || (snpReq != null && - snpReq.snpName.equals(sctx.snapshotName()) && - snpReq.bltNodes.contains(leftNodeId))) { + snpReq.snapshotName().equals(sctx.snapshotName()) && + snpReq.nodes().contains(leftNodeId))) { sctx.acceptException(new ClusterTopologyCheckedException("Snapshot operation interrupted. " + "One of baseline nodes left the cluster: " + leftNodeId)); } @@ -567,7 +565,7 @@ private IgniteInternalFuture initLocalSnapshotStartSt "Another snapshot operation in progress [req=" + req + ", curr=" + clusterSnpReq + ']')); } - Set leftNodes = new HashSet<>(req.bltNodes); + Set leftNodes = new HashSet<>(req.nodes()); leftNodes.removeAll(F.viewReadOnly(cctx.discovery().serverNodes(AffinityTopologyVersion.NONE), F.node2id())); @@ -576,7 +574,9 @@ private IgniteInternalFuture initLocalSnapshotStartSt "prior to snapshot operation start: " + leftNodes)); } - Set leftGrps = new HashSet<>(req.grpIds); + List grpIds = new ArrayList<>(F.viewReadOnly(req.groups(), CU::cacheId)); + + Set leftGrps = new HashSet<>(grpIds); leftGrps.removeAll(cctx.cache().cacheGroupDescriptors().keySet()); if (!leftGrps.isEmpty()) { @@ -588,7 +588,7 @@ private IgniteInternalFuture initLocalSnapshotStartSt // Prepare collection of pairs group and appropriate cache partition to be snapshot. // Cache group context may be 'null' on some nodes e.g. a node filter is set. - for (Integer grpId : req.grpIds) { + for (Integer grpId : grpIds) { if (cctx.cache().cacheGroup(grpId) == null) continue; @@ -600,10 +600,10 @@ private IgniteInternalFuture initLocalSnapshotStartSt if (parts.isEmpty()) task0 = new GridFinishedFuture<>(Collections.emptySet()); else { - task0 = registerSnapshotTask(req.snpName, - req.srcNodeId, + task0 = registerSnapshotTask(req.snapshotName(), + req.operNodeId(), parts, - locSndrFactory.apply(req.snpName)); + locSndrFactory.apply(req.snapshotName())); clusterSnpReq = req; } @@ -613,11 +613,11 @@ private IgniteInternalFuture initLocalSnapshotStartSt throw F.wrap(fut.error()); try { - Set blts = req.bltNodes.stream() + Set blts = req.nodes().stream() .map(n -> cctx.discovery().node(n).consistentId().toString()) .collect(Collectors.toSet()); - File smf = new File(snapshotLocalDir(req.snpName), snapshotMetaFileName(cctx.localNode().consistentId().toString())); + File smf = new File(snapshotLocalDir(req.snapshotName()), snapshotMetaFileName(cctx.localNode().consistentId().toString())); if (smf.exists()) throw new GridClosureException(new IgniteException("Snapshot metafile must not exist: " + smf.getAbsolutePath())); @@ -626,12 +626,12 @@ private IgniteInternalFuture initLocalSnapshotStartSt try (OutputStream out = new BufferedOutputStream(new FileOutputStream(smf))) { U.marshal(marsh, - new SnapshotMetadata(req.rqId, - req.snpName, + new SnapshotMetadata(req.requestId(), + req.snapshotName(), cctx.localNode().consistentId().toString(), pdsSettings.folderName(), cctx.gridConfig().getDataStorageConfiguration().getPageSize(), - req.grpIds, + grpIds, blts, fut.result()), out); @@ -660,7 +660,7 @@ private void processLocalSnapshotStartStageResult(UUID id, Map e instanceof IgniteFutureCancelledCheckedException); - if (snpReq == null || !snpReq.rqId.equals(id)) { + if (snpReq == null || !snpReq.requestId().equals(id)) { synchronized (snpOpMux) { if (clusterSnpFut != null && clusterSnpFut.rqId.equals(id)) { if (cancelled) { @@ -679,18 +679,18 @@ private void processLocalSnapshotStartStageResult(UUID id, Map missed = new HashSet<>(snpReq.bltNodes); + Set missed = new HashSet<>(snpReq.nodes()); missed.removeAll(res.keySet()); missed.removeAll(err.keySet()); if (cancelled) { - snpReq.err = new IgniteFutureCancelledCheckedException("Execution of snapshot tasks " + - "has been cancelled by external process [err=" + err + ", missed=" + missed + ']'); + snpReq.error(new IgniteFutureCancelledCheckedException("Execution of snapshot tasks " + + "has been cancelled by external process [err=" + err + ", missed=" + missed + ']')); } else if (!F.isEmpty(err) || !missed.isEmpty()) { - snpReq.err = new IgniteCheckedException("Execution of local snapshot tasks fails or them haven't been executed " + + snpReq.error(new IgniteCheckedException("Execution of local snapshot tasks fails or them haven't been executed " + "due to some of nodes left the cluster. Uncompleted snapshot will be deleted " + - "[err=" + err + ", missed=" + missed + ']'); + "[err=" + err + ", missed=" + missed + ']')); } endSnpProc.start(UUID.randomUUID(), snpReq); @@ -706,8 +706,8 @@ private IgniteInternalFuture initLocalSnapshotEndStag return new GridFinishedFuture<>(new SnapshotOperationResponse()); try { - if (req.err != null) - deleteSnapshot(snapshotLocalDir(req.snpName), pdsSettings.folderName()); + if (req.error() != null) + deleteSnapshot(snapshotLocalDir(req.snapshotName()), pdsSettings.folderName()); removeLastMetaStorageKey(); } @@ -729,26 +729,26 @@ private void processLocalSnapshotEndStageResult(UUID id, Map endFail = new HashSet<>(snpReq.bltNodes); + Set endFail = new HashSet<>(snpReq.nodes()); endFail.removeAll(res.keySet()); clusterSnpReq = null; synchronized (snpOpMux) { if (clusterSnpFut != null) { - if (endFail.isEmpty() && snpReq.err == null) { + if (endFail.isEmpty() && snpReq.error() == null) { clusterSnpFut.onDone(); if (log.isInfoEnabled()) log.info(SNAPSHOT_FINISHED_MSG + snpReq); } - else if (snpReq.err == null) { + else if (snpReq.error() == null) { clusterSnpFut.onDone(new IgniteCheckedException("Snapshot creation has been finished with an error. " + "Local snapshot tasks may not finished completely or finalizing results fails " + "[fail=" + endFail + ", err=" + err + ']')); } else - clusterSnpFut.onDone(snpReq.err); + clusterSnpFut.onDone(snpReq.error()); clusterSnpFut = null; } @@ -1078,10 +1078,10 @@ public List readSnapshotMetadatas(String snpName) { lastSeenSnpFut = snpFut0; } - List grps = cctx.cache().persistentGroups().stream() + List grps = cctx.cache().persistentGroups().stream() .filter(g -> cctx.cache().cacheType(g.cacheOrGroupName()) == CacheType.USER) .filter(g -> !g.config().isEncryptionEnabled()) - .map(CacheGroupDescriptor::groupId) + .map(CacheGroupDescriptor::cacheOrGroupName) .collect(Collectors.toList()); List srvNodes = cctx.discovery().serverNodes(AffinityTopologyVersion.NONE); @@ -1099,7 +1099,8 @@ public List readSnapshotMetadatas(String snpName) { grps, new HashSet<>(F.viewReadOnly(srvNodes, F.node2id(), - (node) -> CU.baselineNode(node, clusterState))))); + (node) -> CU.baselineNode(node, clusterState))) + )); String msg = "Cluster-wide snapshot operation started [snpName=" + name + ", grps=" + grps + ']'; @@ -1176,13 +1177,13 @@ public static boolean isSnapshotOperation(DiscoveryEvent evt) { SnapshotOperationRequest snpReq = clusterSnpReq; - SnapshotFutureTask task = locSnpTasks.get(snpReq.snpName); + SnapshotFutureTask task = locSnpTasks.get(snpReq.snapshotName()); if (task == null) return; if (task.start()) { - cctx.database().forceCheckpoint(String.format("Start snapshot operation: %s", snpReq.snpName)); + cctx.database().forceCheckpoint(String.format("Start snapshot operation: %s", snpReq.snapshotName())); // Schedule task on a checkpoint and wait when it starts. try { @@ -1840,49 +1841,6 @@ public LocalSnapshotSender(String snpName) { } } - /** Snapshot start request for {@link DistributedProcess} initiate message. */ - private static class SnapshotOperationRequest implements Serializable { - /** Serial version uid. */ - private static final long serialVersionUID = 0L; - - /** Unique snapshot request id. */ - private final UUID rqId; - - /** Source node id which trigger request. */ - private final UUID srcNodeId; - - /** Snapshot name. */ - private final String snpName; - - /** The list of cache groups to include into snapshot. */ - @GridToStringInclude - private final List grpIds; - - /** The list of affected by snapshot operation baseline nodes. */ - @GridToStringInclude - private final Set bltNodes; - - /** Exception occurred during snapshot operation processing. */ - private volatile IgniteCheckedException err; - - /** - * @param snpName Snapshot name. - * @param grpIds Cache groups to include into snapshot. - */ - public SnapshotOperationRequest(UUID rqId, UUID srcNodeId, String snpName, List grpIds, Set bltNodes) { - this.rqId = rqId; - this.srcNodeId = srcNodeId; - this.snpName = snpName; - this.grpIds = grpIds; - this.bltNodes = bltNodes; - } - - /** {@inheritDoc} */ - @Override public String toString() { - return S.toString(SnapshotOperationRequest.class, this); - } - } - /** */ private static class SnapshotOperationResponse implements Serializable { /** Serial version uid. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java similarity index 64% rename from modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java rename to modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index 98d805f243e3f9..935cd76e25cd54 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -21,12 +21,14 @@ import java.util.Collection; import java.util.Set; import java.util.UUID; +import org.apache.ignite.internal.util.distributed.DistributedProcess; +import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; /** - * Request to prepare cache group restore from the snapshot. + * Snapshot operation start request for {@link DistributedProcess} initiate message. */ -public class SnapshotRestoreRequest implements Serializable { +public class SnapshotOperationRequest implements Serializable { /** Serial version uid. */ private static final long serialVersionUID = 0L; @@ -37,33 +39,38 @@ public class SnapshotRestoreRequest implements Serializable { private final String snpName; /** Baseline node IDs that must be alive to complete the operation. */ + @GridToStringInclude private final Set nodes; - /** List of cache group names to restore from the snapshot. */ + /** List of cache group names. */ + @GridToStringInclude private final Collection grps; - /** Node ID from which to update the binary metadata. */ - private final UUID updateMetaNodeId; + /** Operational node ID. */ + private final UUID opNodeId; + + /** Exception occurred during snapshot operation processing. */ + private volatile Throwable err; /** * @param reqId Request ID. + * @param opNodeId Operational node ID. * @param snpName Snapshot name. + * @param grps List of cache group names. * @param nodes Baseline node IDs that must be alive to complete the operation. - * @param grps List of cache group names to restore from the snapshot. - * @param updateMetaNodeId Node ID from which to update the binary metadata. */ - public SnapshotRestoreRequest( + public SnapshotOperationRequest( UUID reqId, + UUID opNodeId, String snpName, - Set nodes, Collection grps, - UUID updateMetaNodeId + Set nodes ) { this.reqId = reqId; + this.opNodeId = opNodeId; this.snpName = snpName; - this.nodes = nodes; this.grps = grps; - this.updateMetaNodeId = updateMetaNodeId; + this.nodes = nodes; } /** @@ -81,7 +88,7 @@ public String snapshotName() { } /** - * @return List of cache group names to restore from the snapshot. + * @return List of cache group names. */ public Collection groups() { return grps; @@ -95,14 +102,28 @@ public Set nodes() { } /** - * @return Node ID from which to update the binary metadata. + * @return Operational node ID. + */ + public UUID operNodeId() { + return opNodeId; + } + + /** + * @return Exception occurred during snapshot operation processing. + */ + public Throwable error() { + return err; + } + + /** + * @param err Exception occurred during snapshot operation processing. */ - public UUID updateMetaNodeId() { - return updateMetaNodeId; + public void error(Throwable err) { + this.err = err; } /** {@inheritDoc} */ @Override public String toString() { - return S.toString(SnapshotRestoreRequest.class, this); + return S.toString(SnapshotOperationRequest.class, this); } } 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 5e78ca709e7a30..bc854ece7ed44e 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 @@ -84,7 +84,7 @@ public class SnapshotRestoreProcess { private final GridKernalContext ctx; /** Cache group restore prepare phase. */ - private final DistributedProcess> prepareRestoreProc; + private final DistributedProcess> prepareRestoreProc; /** Cache group restore cache start phase. */ private final DistributedProcess cacheStartProc; @@ -207,8 +207,8 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - SnapshotRestoreRequest req = - new SnapshotRestoreRequest(fut.rqId, snpName, dataNodes, cacheGrpNames, F.first(dataNodes)); + SnapshotOperationRequest req = + new SnapshotOperationRequest(fut.rqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); prepareRestoreProc.start(req.requestId(), req); } @@ -360,7 +360,7 @@ private void ensureCacheAbsent(String name) { * @param req Request to prepare cache group restore from the snapshot. * @return Result future. */ - private IgniteInternalFuture> prepare(SnapshotRestoreRequest req) { + private IgniteInternalFuture> prepare(SnapshotOperationRequest req) { if (ctx.clientNode()) return new GridFinishedFuture<>(); @@ -412,7 +412,7 @@ private IgniteInternalFuture> prepare(SnapshotRestore opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); - restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.updateMetaNodeId()), stopChecker, errHnd) + restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.operNodeId()), stopChecker, errHnd) .thenAccept(res -> { Throwable err = opCtx.err.get(); @@ -513,7 +513,7 @@ private CompletableFuture restoreAsync( * @return Snapshot restore operation context. * @throws IgniteCheckedException If failed. */ - private SnapshotRestoreContext prepareContext(SnapshotRestoreRequest req) throws IgniteCheckedException { + private SnapshotRestoreContext prepareContext(SnapshotOperationRequest req) throws IgniteCheckedException { if (opCtx != null) { throw new IgniteCheckedException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); @@ -804,7 +804,7 @@ private static class SnapshotRestoreContext { * @param dirs List of cache group names to restore from the snapshot. * @param cfgs Cache ID to configuration mapping. */ - protected SnapshotRestoreContext(SnapshotRestoreRequest req, Collection dirs, + protected SnapshotRestoreContext(SnapshotOperationRequest req, Collection dirs, Map cfgs) { reqId = req.requestId(); snpName = req.snapshotName(); From 2233121e600ea86a44e4e8a4a89118e97aacb82e Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 6 Apr 2021 15:33:34 +0300 Subject: [PATCH 75/98] IGNITE-13805 (minor) Code cleanup. --- .../cache/persistence/snapshot/IgniteSnapshotManager.java | 4 ++-- .../cache/persistence/snapshot/SnapshotRestoreProcess.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 5f0f49561efbbb..50d94fecfe6df2 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 @@ -875,7 +875,7 @@ public IgniteInternalFuture checkSnapshot(String name) { if (f0.error() == null) { Map> metas = f0.result(); - runSnapshotVerfification(metas).listen(f1 -> { + runSnapshotVerification(metas).listen(f1 -> { if (f1.error() == null) res.onDone(f1.result()); else if (f1.error() instanceof IgniteSnapshotVerifyException) @@ -917,7 +917,7 @@ IgniteInternalFuture>> collectSnapshotMe * @param metas Nodes snapshot metadata. * @return Future with the verification results. */ - IgniteInternalFuture runSnapshotVerfification(Map> metas) { + IgniteInternalFuture runSnapshotVerification(Map> metas) { GridKernalContext kctx0 = cctx.kernalContext(); kctx0.task().setThreadContext(TC_SKIP_AUTH, true); 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 bc854ece7ed44e..a50012ec606736 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 @@ -187,7 +187,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - ctx.cache().context().snapshotMgr().runSnapshotVerfification(metas).listen( + ctx.cache().context().snapshotMgr().runSnapshotVerification(metas).listen( f0 -> { if (f0.error() != null) { finishProcess(fut.rqId, f0.error()); From 4747d2c4f5cd5f45e1d97b977f7fc66b6bcf1f12 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 6 Apr 2021 18:11:21 +0300 Subject: [PATCH 76/98] IGNITE-13805 (minor) Test code cleanup. --- .../IgniteClusterSnapshotRestoreSelfTest.java | 46 ++++++------------- 1 file changed, 13 insertions(+), 33 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 33aac59c5a252c..34e12b676f86eb 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 @@ -46,20 +46,17 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; +import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.Nullable; import org.junit.Test; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_GRP_DIR_PREFIX; -import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; 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.runAsync; @@ -132,7 +129,7 @@ public void testCacheStartFailOnNodeLeft() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, null); - ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + ensureCacheDirEmpty(2, dfltCacheCfg); } /** @throws Exception If failed. */ @@ -160,7 +157,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); - // remove metadata + // Remove metadata. int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); ignite.context().cacheObjects().removeType(typeId); @@ -217,7 +214,7 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { awaitPartitionMapExchange(); - // remove metadata + // Remove metadata. int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); ignite.context().cacheObjects().removeType(typeId); @@ -312,7 +309,7 @@ public void testIncompatibleMetasUpdate() throws Exception { fut.get(TIMEOUT); - // Ensure that existing type has been updated + // Ensure that existing type has been updated. BinaryType type = ignite.context().cacheObjects().metadata(typeId); assertTrue(type.fieldNames().contains("name")); @@ -342,7 +339,7 @@ public void testIncompatibleMetasUpdate() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut0.get(TIMEOUT), BinaryObjectException.class, null); - ensureCacheDirEmpty(2, dfltCacheCfg.getName()); + ensureCacheDirEmpty(2, dfltCacheCfg); for (int i = 0; i < CACHE_KEYS_RANGE; i++) assertEquals(objs[i], cache1.get(i)); @@ -451,7 +448,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { "Required node has left the cluster" ); - ensureCacheDirEmpty(3, dfltCacheCfg.getName()); + ensureCacheDirEmpty(3, dfltCacheCfg); fut0.get(TIMEOUT); @@ -581,7 +578,7 @@ private void checkClusterStateChange( ignite.cluster().state(ClusterState.ACTIVE); - ensureCacheDirEmpty(stopNode ? nodesCnt - 1 : nodesCnt, dfltCacheCfg.getName()); + ensureCacheDirEmpty(stopNode ? nodesCnt - 1 : nodesCnt, dfltCacheCfg); String cacheName = dfltCacheCfg.getName(); @@ -600,10 +597,12 @@ private void checkClusterStateChange( /** * @param nodesCnt Count of nodes. - * @param cacheName Cache name. + * @param ccfg Cache configuration. * @throws IgniteCheckedException if failed. */ - private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCheckedException { + private void ensureCacheDirEmpty(int nodesCnt, CacheConfiguration ccfg) throws IgniteCheckedException { + String cacheName = ccfg.getName(); + for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { IgniteEx grid = grid(nodeIdx); @@ -615,7 +614,7 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh () -> !grid.context().cache().context().snapshotMgr().isRestoring(), TIMEOUT); - File dir = resolveCacheDir(grid, cacheName); + File dir = ((FilePageStoreManager)grid.context().cache().context().pageStore()).cacheWorkDir(ccfg); String errMsg = String.format("%s, dir=%s, exists=%b, files=%s", grid.name(), dir, dir.exists(), Arrays.toString(dir.list())); @@ -624,25 +623,6 @@ private void ensureCacheDirEmpty(int nodesCnt, String cacheName) throws IgniteCh } } - /** - * @param ignite Ignite. - * @param cacheOrGrpName Cache (or group) name. - * @return Local path to the cache directory. - * @throws IgniteCheckedException if failed. - */ - private File resolveCacheDir(IgniteEx ignite, String cacheOrGrpName) throws IgniteCheckedException { - File workDIr = U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_STORE_DIR, false); - - String nodeDirName = ignite.context().pdsFolderResolver().resolveFolders().folderName() + File.separator; - - File cacheDir = new File(workDIr, nodeDirName + CACHE_DIR_PREFIX + cacheOrGrpName); - - if (cacheDir.exists()) - return cacheDir; - - return new File(workDIr, nodeDirName + CACHE_GRP_DIR_PREFIX + cacheOrGrpName); - } - /** * @param nodesCnt Nodes count. * @param keysCnt Number of keys to create. From 24b5d91bbaec6417c233069ec1272acf8f18d2c1 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 7 Apr 2021 00:05:14 +0300 Subject: [PATCH 77/98] IGNITE-13805 (minor) Code cleanup. --- .../persistence/snapshot/SnapshotRestoreProcess.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) 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 a50012ec606736..4fadcb5a5116cf 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 @@ -18,7 +18,6 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import java.io.File; -import java.io.IOException; import java.nio.file.Files; import java.nio.file.Paths; import java.util.ArrayList; @@ -496,8 +495,8 @@ private CompletableFuture restoreAsync( Files.copy(snpFile.toPath(), target.toPath()); } - catch (IgniteInterruptedCheckedException | IOException e) { - errHnd.accept(e); + catch (Throwable t) { + errHnd.accept(t); } }, ctx.cache().context().snapshotMgr().snapshotExecutorService())); } @@ -559,7 +558,7 @@ else if (cacheDir.list().length > 0) { pageStore.readCacheConfigurations(snpCacheDir, cfgsByName); } - Map cfgsById = cfgsByName.isEmpty() ? Collections.emptyMap() : + Map cfgsById = cfgsByName.values().stream().collect(Collectors.toMap(v -> CU.cacheId(v.config().getName()), v -> v)); return new SnapshotRestoreContext(req, cacheDirs, cfgsById); @@ -578,7 +577,7 @@ private void finishPrepare(UUID reqId, Map> res Exception failure = F.first(errs.values()); - assert opCtx0 != null || failure != null : ctx.localNodeId(); + assert opCtx0 != null || failure != null : "Context has not been created on the node " + ctx.localNodeId(); if (opCtx0 == null) { finishProcess(reqId, failure); From 288cf8d9c0ad4e17c4e90a552187e2fc09d50d50 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 14 Apr 2021 14:49:07 +0300 Subject: [PATCH 78/98] IGNITE-13805 (minor) Code cleanup. --- .../processors/cache/ClusterCachesInfo.java | 2 +- .../cache/DynamicCacheChangeBatch.java | 12 ++++++------ .../processors/cache/ExchangeActions.java | 15 ++++++++------- .../processors/cache/GridCacheProcessor.java | 18 ++++++++++-------- .../GridDhtPartitionsExchangeFuture.java | 2 +- .../snapshot/IgniteSnapshotManager.java | 6 +++++- .../snapshot/SnapshotOperationRequest.java | 2 +- .../snapshot/SnapshotRestoreProcess.java | 16 +++++++++------- 8 files changed, 41 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 40a7515fd574f4..14851d8a46fa00 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -629,7 +629,7 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop batch.exchangeActions(exchangeActions); if (!F.isEmpty(batch.topologyNodes())) - exchangeActions.cacheStartTopologySnapshot(batch.topologyNodes()); + exchangeActions.cacheStartRequiredAliveNodes(batch.topologyNodes()); } return res.needExchange; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index dbd97c3f62dacf..a38c006a20eae8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -50,7 +50,7 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { * Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when starting * the cache(s), the whole procedure is rolled back. */ - private Collection topNodes; + private Collection rqNodes; /** Cache updates to be executed on exchange. */ private transient ExchangeActions exchangeActions; @@ -67,14 +67,14 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { /** * @param reqs Requests. - * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails - * when starting the cache(s), the whole procedure is rolled back. + * @param rqNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails + * when starting the cache(s), the whole procedure is rolled back. */ - public DynamicCacheChangeBatch(Collection reqs, @Nullable Collection topNodes) { + public DynamicCacheChangeBatch(Collection reqs, @Nullable Collection rqNodes) { assert !F.isEmpty(reqs) : reqs; this.reqs = reqs; - this.topNodes = topNodes; + this.rqNodes = rqNodes; } /** {@inheritDoc} */ @@ -177,7 +177,7 @@ public void startCaches(boolean startCaches) { * starting the cache(s), the whole procedure is rolled back. */ public Collection topologyNodes() { - return topNodes; + return rqNodes; } /** {@inheritDoc} */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java index a0946150fa0dd1..8736a88d53c667 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ExchangeActions.java @@ -48,7 +48,7 @@ public class ExchangeActions { * Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when starting * the cache(s), the whole procedure is rolled back. */ - private Collection cacheStartTopSnapshot; + private Collection cacheStartRequiredAliveNodes; /** */ private Map cachesToStop; @@ -330,16 +330,17 @@ public boolean cacheGroupStarting(int grpId) { * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when * starting the cache(s), the whole procedure is rolled back. */ - public Collection cacheStartTopologySnapshot() { - return cacheStartTopSnapshot == null ? Collections.emptyList() : cacheStartTopSnapshot; + public Collection cacheStartRequiredAliveNodes() { + return cacheStartRequiredAliveNodes == null ? Collections.emptyList() : cacheStartRequiredAliveNodes; } /** - * @param cacheStartTopSnapshot Server nodes on which a successful start of the cache(s) is required, if any of - * these nodes fails when starting the cache(s), the whole procedure is rolled back. + * @param cacheStartRequiredAliveNodes Server nodes on which a successful start of the cache(s) is required, if any + * of these nodes fails when starting the cache(s), the whole procedure is + * rolled back. */ - public void cacheStartTopologySnapshot(Collection cacheStartTopSnapshot) { - this.cacheStartTopSnapshot = new ArrayList<>(cacheStartTopSnapshot); + public void cacheStartRequiredAliveNodes(Collection cacheStartRequiredAliveNodes) { + this.cacheStartRequiredAliveNodes = new ArrayList<>(cacheStartRequiredAliveNodes); } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index a9678b19c0f92e..9bd48d17dac6f0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3709,6 +3709,7 @@ private IgniteInternalFuture startClientCacheChange( * @param failIfExists Fail if exists flag. * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. + * @param internal Flag indicating that the cache was started internally and not by the user. * @return Future that will be completed when all caches are deployed. */ public IgniteInternalFuture dynamicStartCaches( @@ -3716,7 +3717,7 @@ public IgniteInternalFuture dynamicStartCaches( boolean failIfExists, boolean checkThreadTx, boolean disabledAfterStart, - boolean restoredCache + boolean internal ) { return dynamicStartCachesByStoredConf( ccfgList.stream().map(StoredCacheData::new).collect(toList()), @@ -3724,7 +3725,7 @@ public IgniteInternalFuture dynamicStartCaches( checkThreadTx, disabledAfterStart, null, - restoredCache); + internal); } /** @@ -3735,6 +3736,7 @@ public IgniteInternalFuture dynamicStartCaches( * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. * @param restartId Restart requester id (it'll allow to start this cache only him). + * @param internal Flag indicating that the cache was started internally and not by the user. * @return Future that will be completed when all caches are deployed. */ public IgniteInternalFuture dynamicStartCachesByStoredConf( @@ -3743,10 +3745,10 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean checkThreadTx, boolean disabledAfterStart, IgniteUuid restartId, - boolean restoredCache + boolean internal ) { return dynamicStartCachesByStoredConf(storedCacheDataList, failIfExists, checkThreadTx, disabledAfterStart, - restartId, restoredCache, null); + restartId, internal, null); } /** @@ -4090,13 +4092,13 @@ private Collection initiateCacheChanges( /** * @param reqs Requests. - * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails - * when starting the cache(s), the whole procedure is rolled back. + * @param rqNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails + * when starting the cache(s), the whole procedure is rolled back. * @return Collection of futures. */ private Collection initiateCacheChanges( Collection reqs, - @Nullable Collection topNodes + @Nullable Collection rqNodes ) { Collection res = new ArrayList<>(reqs.size()); @@ -4152,7 +4154,7 @@ private Collection initiateCacheChanges( if (!sndReqs.isEmpty()) { try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, topNodes)); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, rqNodes)); err = checkNodeState(); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 01ccbf337d462c..6906fadeb5c19c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -3834,7 +3834,7 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe return; } - for (UUID nodeId : exchActions.cacheStartTopologySnapshot()) { + for (UUID nodeId : exchActions.cacheStartRequiredAliveNodes()) { ClusterNode node = cctx.discovery().node(nodeId); if (node != 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 50d94fecfe6df2..0241462e034ab7 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 @@ -601,7 +601,7 @@ private IgniteInternalFuture initLocalSnapshotStartSt task0 = new GridFinishedFuture<>(Collections.emptySet()); else { task0 = registerSnapshotTask(req.snapshotName(), - req.operNodeId(), + req.operationalNodeId(), parts, locSndrFactory.apply(req.snapshotName())); @@ -1124,6 +1124,10 @@ public List readSnapshotMetadatas(String snpName) { /** {@inheritDoc} */ @Override public IgniteFuture restoreSnapshot(String name, Collection grpNames) { + 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_"); + A.ensure(!F.isEmpty(grpNames), "List of cache group names cannot be empty."); + return restoreCacheGrpProc.start(name, grpNames); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index 935cd76e25cd54..b8bd5a26a126da 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -104,7 +104,7 @@ public Set nodes() { /** * @return Operational node ID. */ - public UUID operNodeId() { + public UUID operationalNodeId() { return opNodeId; } 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 4fadcb5a5116cf..4bae41b388aa36 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 @@ -126,6 +126,8 @@ public SnapshotRestoreProcess(GridKernalContext ctx) { * @return Future that will be completed when the restore operation is complete and the cache groups are started. */ public IgniteFuture start(String snpName, Collection cacheGrpNames) { + UUID reqId = UUID.randomUUID(); + try { if (ctx.clientNode()) throw new IgniteException(OP_REJECT_MSG + "Client and daemon nodes can not perform this operation."); @@ -148,7 +150,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames if (isRestoring() || fut != null) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - fut = new ClusterSnapshotFuture(UUID.randomUUID(), snpName); + fut = new ClusterSnapshotFuture(reqId, snpName); } } catch (IgniteException e) { return new IgniteFinishedFutureImpl<>(e); @@ -157,7 +159,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - finishProcess(fut.rqId, f.error()); + finishProcess(reqId, f.error()); return; } @@ -180,7 +182,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - finishProcess(fut.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + finishProcess(reqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); return; @@ -189,7 +191,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().runSnapshotVerification(metas).listen( f0 -> { if (f0.error() != null) { - finishProcess(fut.rqId, f0.error()); + finishProcess(reqId, f0.error()); return; } @@ -201,13 +203,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - finishProcess(fut.rqId, new IgniteException(sb.toString())); + finishProcess(reqId, new IgniteException(sb.toString())); return; } SnapshotOperationRequest req = - new SnapshotOperationRequest(fut.rqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); + new SnapshotOperationRequest(reqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); prepareRestoreProc.start(req.requestId(), req); } @@ -411,7 +413,7 @@ private IgniteInternalFuture> prepare(SnapshotOperati opCtx0.stopFut = new IgniteFutureImpl<>(retFut.chain(f -> null)); - restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.operNodeId()), stopChecker, errHnd) + restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.operationalNodeId()), stopChecker, errHnd) .thenAccept(res -> { Throwable err = opCtx.err.get(); From b955d5c0bbe9cf1f20b15ecdb0bdffd143fddd00 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 14 Apr 2021 15:53:38 +0300 Subject: [PATCH 79/98] IGNITE-13805 Check required nodes alive. --- .../snapshot/SnapshotRestoreProcess.java | 37 +++++++++++++++---- .../IgniteClusterSnapshotRestoreSelfTest.java | 26 +++++++++++-- 2 files changed, 52 insertions(+), 11 deletions(-) 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 4bae41b388aa36..4c49871acb1ab3 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 @@ -47,6 +47,7 @@ import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.NodeStoppingException; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.StoredCacheData; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; @@ -126,7 +127,7 @@ public SnapshotRestoreProcess(GridKernalContext ctx) { * @return Future that will be completed when the restore operation is complete and the cache groups are started. */ public IgniteFuture start(String snpName, Collection cacheGrpNames) { - UUID reqId = UUID.randomUUID(); + ClusterSnapshotFuture fut0; try { if (ctx.clientNode()) @@ -150,7 +151,9 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames if (isRestoring() || fut != null) throw new IgniteException(OP_REJECT_MSG + "The previous snapshot restore operation was not completed."); - fut = new ClusterSnapshotFuture(reqId, snpName); + fut = new ClusterSnapshotFuture(UUID.randomUUID(), snpName); + + fut0 = fut; } } catch (IgniteException e) { return new IgniteFinishedFutureImpl<>(e); @@ -159,7 +162,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames ctx.cache().context().snapshotMgr().collectSnapshotMetadata(snpName).listen( f -> { if (f.error() != null) { - finishProcess(reqId, f.error()); + finishProcess(fut0.rqId, f.error()); return; } @@ -167,6 +170,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames Set dataNodes = new HashSet<>(); Map> metas = f.result(); Map reqGrpIds = cacheGrpNames.stream().collect(Collectors.toMap(CU::cacheId, v -> v)); + Set snpBltNodes = null; for (Map.Entry> entry : metas.entrySet()) { SnapshotMetadata meta = F.first(entry.getValue()); @@ -176,22 +180,39 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames if (!entry.getKey().consistentId().equals(meta.consistentId())) continue; + if (snpBltNodes == null) + snpBltNodes = new HashSet<>(meta.baselineNodes()); + dataNodes.add(entry.getKey().id()); reqGrpIds.keySet().removeAll(meta.partitions().keySet()); } if (!reqGrpIds.isEmpty()) { - finishProcess(reqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + + finishProcess(fut0.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); return; } + Collection bltNodes = F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), + node -> node.consistentId().toString(), (node) -> CU.baselineNode(node, ctx.state().clusterState())); + + assert !F.isEmpty(snpBltNodes); + + snpBltNodes.removeAll(bltNodes); + + if (!snpBltNodes.isEmpty()) { + finishProcess(fut0.rqId, new IgniteIllegalStateException(OP_REJECT_MSG + "Some nodes required to " + + "restore a cache group are missing [nodeId(s)=" + snpBltNodes + ", snapshot=" + snpName + ']')); + + return; + } + ctx.cache().context().snapshotMgr().runSnapshotVerification(metas).listen( f0 -> { if (f0.error() != null) { - finishProcess(reqId, f0.error()); + finishProcess(fut0.rqId, f0.error()); return; } @@ -203,13 +224,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames res.print(sb::append, true); - finishProcess(reqId, new IgniteException(sb.toString())); + finishProcess(fut0.rqId, new IgniteException(sb.toString())); return; } SnapshotOperationRequest req = - new SnapshotOperationRequest(reqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); + new SnapshotOperationRequest(fut0.rqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); prepareRestoreProc.start(req.requestId(), req); } @@ -217,7 +238,7 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } ); - return new IgniteFutureImpl<>(fut); + return new IgniteFutureImpl<>(fut0); } /** 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 34e12b676f86eb..b6cd63b138a082 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 @@ -28,6 +28,7 @@ import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteSnapshot; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; @@ -72,10 +73,10 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe private static final String BIN_TYPE_NAME = "customType"; /** Static cache configurations. */ - protected CacheConfiguration[] cacheCfgs; + private CacheConfiguration[] cacheCfgs; /** Cache value builder. */ - protected Function valBuilder = new IndexedValueBuilder(); + private Function valBuilder = new IndexedValueBuilder(); /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { @@ -191,7 +192,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception /** @throws Exception If failed. */ @Test - public void testClusterSnapshotRestoreDiffTopology() throws Exception { + public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { int nodesCnt = 4; int keysCnt = 10_000; @@ -234,6 +235,25 @@ public void testClusterSnapshotRestoreDiffTopology() throws Exception { checkCacheKeys(cache, keysCnt); } + /** @throws Exception If failed. */ + @Test + public void testClusterSnapshotRestoreOnSmallerTopology() throws Exception { + int keysCnt = 10_000; + + startGridsWithSnapshot(2, keysCnt, true); + + stopGrid(1); + + resetBaselineTopology(); + + IgniteFuture fut = + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), IgniteIllegalStateException.class, null); + + ensureCacheDirEmpty(1, dfltCacheCfg); + } + /** @throws Exception If failed. */ @Test public void testRestoreSharedCacheGroup() throws Exception { From dc32d9920b5487a35c24782de341bd983ab781ad Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 14 Apr 2021 16:16:45 +0300 Subject: [PATCH 80/98] IGNITE-13805 (minor) Test code cleanup. --- .../snapshot/AbstractSnapshotSelfTest.java | 3 - .../IgniteClusterSnapshotRestoreSelfTest.java | 120 +++++++----------- 2 files changed, 46 insertions(+), 77 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index d6bb70975d4c3b..7ad06f448434a7 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -124,9 +124,6 @@ public abstract class AbstractSnapshotSelfTest extends GridCommonAbstractTest { discoSpi.setIpFinder(((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder()); - if (dfltCacheCfg != null) - cfg.setCacheConfiguration(dfltCacheCfg); - return cfg.setConsistentId(igniteInstanceName) .setCommunicationSpi(new TestRecordingCommunicationSpi()) .setDataStorageConfiguration(new DataStorageConfiguration() 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 b6cd63b138a082..9cf892d8895d95 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 @@ -41,6 +41,7 @@ import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.TestRecordingCommunicationSpi; @@ -51,6 +52,7 @@ import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.IgniteSpiException; @@ -72,6 +74,15 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe /** Binary type name. */ private static final String BIN_TYPE_NAME = "customType"; + /** Cache 1 name. */ + private static final String CACHE1 = "cache1"; + + /** Cache 2 name. */ + private static final String CACHE2 = "cache2"; + + /** Default shared cache group name. */ + private static final String SHARED_GRP = "shared"; + /** Static cache configurations. */ private CacheConfiguration[] cacheCfgs; @@ -130,7 +141,7 @@ public void testCacheStartFailOnNodeLeft() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, null); - ensureCacheDirEmpty(2, dfltCacheCfg); + ensureCacheDirEmpty(dfltCacheCfg); } /** @throws Exception If failed. */ @@ -251,34 +262,27 @@ public void testClusterSnapshotRestoreOnSmallerTopology() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), IgniteIllegalStateException.class, null); - ensureCacheDirEmpty(1, dfltCacheCfg); + ensureCacheDirEmpty(dfltCacheCfg); } /** @throws Exception If failed. */ @Test public void testRestoreSharedCacheGroup() throws Exception { - String grpName = "shared"; - String cacheName1 = "cache1"; - String cacheName2 = "cache2"; + CacheConfiguration cacheCfg1 = + txCacheConfig(new CacheConfiguration(CACHE1)).setGroupName(SHARED_GRP); - CacheConfiguration cacheCfg1 = txCacheConfig(new CacheConfiguration<>(cacheName1)).setGroupName(grpName); - CacheConfiguration cacheCfg2 = txCacheConfig(new CacheConfiguration<>(cacheName2)).setGroupName(grpName); + CacheConfiguration cacheCfg2 = + txCacheConfig(new CacheConfiguration(CACHE2)).setGroupName(SHARED_GRP); - cacheCfgs = new CacheConfiguration[] {cacheCfg1, cacheCfg2}; + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, cacheCfg1, cacheCfg2); - IgniteEx ignite = startGrids(2); + cacheCfgs = new CacheConfiguration[] {cacheCfg1, cacheCfg2}; ignite.cluster().state(ClusterState.ACTIVE); - IgniteCache cache1 = ignite.cache(cacheName1); - putKeys(cache1, 0, CACHE_KEYS_RANGE); - - IgniteCache cache2 = ignite.cache(cacheName2); - putKeys(cache2, 0, CACHE_KEYS_RANGE); - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - cache1.destroy(); + ignite.cache(CACHE1).destroy(); awaitPartitionMapExchange(); @@ -286,19 +290,19 @@ public void testRestoreSharedCacheGroup() throws Exception { GridTestUtils.assertThrowsAnyCause( log, - () -> snp.restoreSnapshot(SNAPSHOT_NAME, Arrays.asList(cacheName1, cacheName2)).get(TIMEOUT), + () -> snp.restoreSnapshot(SNAPSHOT_NAME, Arrays.asList(CACHE1, CACHE2)).get(TIMEOUT), IllegalArgumentException.class, "Cache group(s) was not found in the snapshot" ); - cache2.destroy(); + ignite.cache(CACHE2).destroy(); awaitPartitionMapExchange(); - snp.restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(grpName)).get(TIMEOUT); + snp.restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(SHARED_GRP)).get(TIMEOUT); - checkCacheKeys(ignite.cache(cacheName1), CACHE_KEYS_RANGE); - checkCacheKeys(ignite.cache(cacheName2), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -359,7 +363,7 @@ public void testIncompatibleMetasUpdate() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut0.get(TIMEOUT), BinaryObjectException.class, null); - ensureCacheDirEmpty(2, dfltCacheCfg); + ensureCacheDirEmpty(dfltCacheCfg); for (int i = 0; i < CACHE_KEYS_RANGE; i++) assertEquals(objs[i], cache1.get(i)); @@ -411,26 +415,23 @@ private void checkCacheStartWithTheSameName( Class expCls, String expMsg ) throws Exception { - String grpName = "shared"; - String cacheName = "cache1"; - - dfltCacheCfg = txCacheConfig(new CacheConfiguration(cacheName)).setGroupName(grpName); + dfltCacheCfg = txCacheConfig(new CacheConfiguration(CACHE1)).setGroupName(SHARED_GRP); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, procType, grpName); + IgniteFuture fut = waitForBlockOnRestore(spi, procType, SHARED_GRP); - GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(grpName), IgniteCheckedException.class, null); + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(SHARED_GRP), IgniteCheckedException.class, null); - GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(cacheName), expCls, expMsg); + GridTestUtils.assertThrowsAnyCause(log, () -> ignite.createCache(CACHE1), expCls, expMsg); spi.stopBlock(); fut.get(TIMEOUT); - checkCacheKeys(grid(0).cache(cacheName), CACHE_KEYS_RANGE); + checkCacheKeys(grid(0).cache(CACHE1), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -468,13 +469,11 @@ private void checkTopologyChange(boolean stopNode) throws Exception { "Required node has left the cluster" ); - ensureCacheDirEmpty(3, dfltCacheCfg); - fut0.get(TIMEOUT); awaitPartitionMapExchange(); - dfltCacheCfg = null; + ensureCacheDirEmpty(dfltCacheCfg); GridTestUtils.assertThrowsAnyCause( log, @@ -551,25 +550,7 @@ private void checkClusterStateChange( @Nullable Class exCls, @Nullable String expMsg ) throws Exception { - checkClusterStateChange(state, procType, exCls, expMsg, false); - } - - /** - * @param state Cluster state. - * @param procType The type of distributed process on which communication is blocked. - * @param exCls Expected exception class. - * @param expMsg Expected exception message. - * @param stopNode Stop node flag. - * @throws Exception if failed. - */ - private void checkClusterStateChange( - ClusterState state, - DistributedProcessType procType, - @Nullable Class exCls, - @Nullable String expMsg, - boolean stopNode - ) throws Exception { - int nodesCnt = stopNode ? 3 : 2; + int nodesCnt = 2; Ignite ignite = startGridsWithSnapshot(nodesCnt, CACHE_KEYS_RANGE, true); @@ -579,10 +560,7 @@ private void checkClusterStateChange( ignite.cluster().state(state); - if (stopNode) - stopGrid(nodesCnt - 1); - else - spi.stopBlock(); + spi.stopBlock(); if (exCls == null) { fut.get(TIMEOUT); @@ -598,46 +576,40 @@ private void checkClusterStateChange( ignite.cluster().state(ClusterState.ACTIVE); - ensureCacheDirEmpty(stopNode ? nodesCnt - 1 : nodesCnt, dfltCacheCfg); + ensureCacheDirEmpty(dfltCacheCfg); String cacheName = dfltCacheCfg.getName(); - if (stopNode) { - dfltCacheCfg = null; - - startGrid(nodesCnt - 1); - - resetBaselineTopology(); - } - grid(nodesCnt - 1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); checkCacheKeys(ignite.cache(cacheName), CACHE_KEYS_RANGE); } /** - * @param nodesCnt Count of nodes. * @param ccfg Cache configuration. * @throws IgniteCheckedException if failed. */ - private void ensureCacheDirEmpty(int nodesCnt, CacheConfiguration ccfg) throws IgniteCheckedException { + private void ensureCacheDirEmpty(CacheConfiguration ccfg) throws IgniteCheckedException { String cacheName = ccfg.getName(); - for (int nodeIdx = 0; nodeIdx < nodesCnt; nodeIdx++) { - IgniteEx grid = grid(nodeIdx); + for (Ignite ignite : G.allGrids()) { + GridKernalContext kctx = ((IgniteEx)ignite).context(); + + if (kctx.clientNode()) + continue; - CacheGroupDescriptor desc = grid.context().cache().cacheGroupDescriptors().get(CU.cacheId(cacheName)); + CacheGroupDescriptor desc = kctx.cache().cacheGroupDescriptors().get(CU.cacheId(cacheName)); - assertNull("nodeIdx=" + nodeIdx + ", cache=" + cacheName, desc); + assertNull("nodeId=" + kctx.localNodeId() + ", cache=" + cacheName, desc); GridTestUtils.waitForCondition( - () -> !grid.context().cache().context().snapshotMgr().isRestoring(), + () -> !kctx.cache().context().snapshotMgr().isRestoring(), TIMEOUT); - File dir = ((FilePageStoreManager)grid.context().cache().context().pageStore()).cacheWorkDir(ccfg); + File dir = ((FilePageStoreManager)kctx.cache().context().pageStore()).cacheWorkDir(ccfg); String errMsg = String.format("%s, dir=%s, exists=%b, files=%s", - grid.name(), dir, dir.exists(), Arrays.toString(dir.list())); + ignite.name(), dir, dir.exists(), Arrays.toString(dir.list())); assertTrue(errMsg, !dir.exists() || dir.list().length == 0); } From 86011cbc356d96c4793bf2eb55bfa03e06fb1eb7 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 15 Apr 2021 15:37:08 +0300 Subject: [PATCH 81/98] IGNITE-13805 Use temp dir to copy files on prepare. Cleanup temp dirs on startup. --- .../snapshot/IgniteSnapshotManager.java | 2 + .../snapshot/SnapshotRestoreProcess.java | 94 +++++++++++++++---- 2 files changed, 80 insertions(+), 16 deletions(-) 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 0241462e034ab7..95403770f44c8c 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 @@ -1145,6 +1145,8 @@ public List readSnapshotMetadatas(String snpName) { /** {@inheritDoc} */ @Override public void onReadyForRead(ReadOnlyMetastorage metaStorage) throws IgniteCheckedException { + restoreCacheGrpProc.cleanup(); + // Snapshot which has not been completed due to the local node crashed must be deleted. String snpName = (String)metaStorage.read(SNP_RUNNING_KEY); 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 4c49871acb1ab3..0c0671935d3ef2 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 @@ -20,6 +20,7 @@ import java.io.File; import java.nio.file.Files; import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -80,6 +81,9 @@ public class SnapshotRestoreProcess { /** Reject operation message. */ private static final String OP_REJECT_MSG = "Cache group restore operation was rejected. "; + /** Temporary cache directory prefix. */ + private static final String TMP_CACHE_DIR_PREFIX = ".tmp.snp.restore."; + /** Kernal context. */ private final GridKernalContext ctx; @@ -119,6 +123,24 @@ public SnapshotRestoreProcess(GridKernalContext ctx) { ctx, RESTORE_CACHE_GROUP_SNAPSHOT_ROLLBACK, this::rollback, this::finishRollback); } + /** + * Cleanup temporary directories if any exists. + * + * @throws IgniteCheckedException If it was not possible to delete some temporary directory. + */ + protected void cleanup() throws IgniteCheckedException { + FilePageStoreManager pageStore = (FilePageStoreManager)ctx.cache().context().pageStore(); + + File dbDir = pageStore.workDir(); + + for (File dir : dbDir.listFiles(dir -> dir.isDirectory() && dir.getName().startsWith(TMP_CACHE_DIR_PREFIX))) { + if (!U.delete(dir)) { + throw new IgniteCheckedException("Unable to remove temporary directory, " + + "try deleting it manually [dir=" + dir + ']'); + } + } + } + /** * Start cache group restore operation. * @@ -436,16 +458,24 @@ private IgniteInternalFuture> prepare(SnapshotOperati restoreAsync(opCtx0.snpName, opCtx0.dirs, ctx.localNodeId().equals(req.operationalNodeId()), stopChecker, errHnd) .thenAccept(res -> { - Throwable err = opCtx.err.get(); + try { + Throwable err = opCtx.err.get(); + + if (err != null) + throw err; - if (err != null) { + for (File src : opCtx0.dirs) + Files.move(formatTmpDirName(src).toPath(), src.toPath(), StandardCopyOption.ATOMIC_MOVE); + } catch (Throwable t) { log.error("Unable to restore cache group(s) from the snapshot " + - "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', err); + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', t); - retFut.onDone(err); + retFut.onDone(t); + + return; } - else - retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); + + retFut.onDone(new ArrayList<>(opCtx.cfgs.values())); }); return retFut; @@ -457,6 +487,14 @@ private IgniteInternalFuture> prepare(SnapshotOperati } } + /** + * @param cacheDir Cache directory. + * @return Temporary directory. + */ + private File formatTmpDirName(File cacheDir) { + return new File(cacheDir.getParent(), TMP_CACHE_DIR_PREFIX + cacheDir.getName()); + } + /** * Copy partition files and update binary metadata. * @@ -493,6 +531,7 @@ private CompletableFuture restoreAsync( } for (File cacheDir : dirs) { + File tmpCacheDir = formatTmpDirName(cacheDir); File snpCacheDir = new File(ctx.cache().context().snapshotMgr().snapshotLocalDir(snpName), Paths.get(databaseRelativePath(pdsFolderName), cacheDir.getName()).toString()); @@ -507,7 +546,7 @@ private CompletableFuture restoreAsync( if (Thread.interrupted()) throw new IgniteInterruptedCheckedException("Thread has been interrupted."); - File target = new File(cacheDir, snpFile.getName()); + File target = new File(tmpCacheDir, snpFile.getName()); if (log.isDebugEnabled()) { log.debug("Copying file from the snapshot " + @@ -560,7 +599,7 @@ private SnapshotRestoreContext prepareContext(SnapshotOperationRequest req) thro Map cfgsByName = new HashMap<>(); FilePageStoreManager pageStore = (FilePageStoreManager)cctx.pageStore(); - // Collect cache configuration(s) and verify cache groups page size. + // Collect the cache configurations and prepare a temporary directory for copying files. for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), meta.folderName())) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); @@ -569,11 +608,28 @@ private SnapshotRestoreContext prepareContext(SnapshotOperationRequest req) thro File cacheDir = pageStore.cacheWorkDir(snpCacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX), grpName); - if (!cacheDir.exists()) - cacheDir.mkdir(); - else if (cacheDir.list().length > 0) { - throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + - "[group=" + grpName + ", dir=" + cacheDir + ']'); + if (cacheDir.exists()) { + if (cacheDir.list().length > 0) { + throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } + + if (!cacheDir.delete()) { + throw new IgniteCheckedException("Unable to remove empty cache directory " + + "[group=" + grpName + ", dir=" + cacheDir + ']'); + } + } + + File tmpCacheDir = formatTmpDirName(cacheDir); + + if (tmpCacheDir.exists()) { + throw new IgniteCheckedException("Unable to restore cache group, temp directory already exists " + + "[group=" + grpName + ", dir=" + tmpCacheDir + ']'); + } + + if (!tmpCacheDir.mkdir()) { + throw new IgniteCheckedException("Unable to restore cache group, cannot create temp directory " + + "[group=" + grpName + ", dir=" + tmpCacheDir + ']'); } cacheDirs.add(cacheDir); @@ -740,10 +796,16 @@ private IgniteInternalFuture rollback(UUID reqId) { IgniteCheckedException ex = null; for (File cacheDir : opCtx0.dirs) { - if (!cacheDir.exists()) - continue; + File tmpCacheDir = formatTmpDirName(cacheDir); + + if (tmpCacheDir.exists() && !U.delete(tmpCacheDir)) { + log.error("Unable to perform rollback routine completely, cannot remove temp directory " + + "[reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dir=" + tmpCacheDir + ']'); + + ex = new IgniteCheckedException("Unable to remove temporary cache directory " + cacheDir); + } - if (!U.delete(cacheDir)) { + if (cacheDir.exists() && !U.delete(cacheDir)) { log.error("Unable to perform rollback routine completely, cannot remove cache directory " + "[reqId=" + reqId + ", snapshot=" + opCtx0.snpName + ", dir=" + cacheDir + ']'); From da28d465b588a7a8901d94e478e48a3d9eb54353 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 15 Apr 2021 17:54:39 +0300 Subject: [PATCH 82/98] IGNITE-13805 Start multithreaded tests. --- .../snapshot/SnapshotRestoreProcess.java | 6 +- .../IgniteClusterSnapshotRestoreSelfTest.java | 80 +++++++++++++++---- 2 files changed, 67 insertions(+), 19 deletions(-) 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 0c0671935d3ef2..aaca8c7a964358 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 @@ -211,8 +211,8 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } if (!reqGrpIds.isEmpty()) { - finishProcess(fut0.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not found in the " + - "snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); + finishProcess(fut0.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not " + + "found in the snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); return; } @@ -658,7 +658,7 @@ private void finishPrepare(UUID reqId, Map> res assert opCtx0 != null || failure != null : "Context has not been created on the node " + ctx.localNodeId(); - if (opCtx0 == null) { + if (opCtx0 == null || !reqId.equals(opCtx0.reqId)) { finishProcess(reqId, failure); return; 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 9cf892d8895d95..e744ea1c8dbab3 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 @@ -23,7 +23,11 @@ import java.util.LinkedHashMap; import java.util.Objects; import java.util.UUID; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import java.util.function.IntSupplier; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -115,6 +119,66 @@ private QueryEntity queryEntity(String typeName) { .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); } + /** @throws Exception If failed. */ + @Test + public void testBasicClusterSnapshotRestore() throws Exception { + int keysCnt = 10_000; + + IgniteEx client = startGridsWithSnapshot(2, keysCnt, true); + + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + + IgniteCache cache = client.cache(dfltCacheCfg.getName()); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, keysCnt); + } + + /** @throws Exception If failed. */ + @Test + public void testStartClusterSnapshotRestoreMultipleThreadsSameNode() throws Exception { + checkStartClusterSnapshotRestoreMultithreaded(() -> 0); + } + + /** @throws Exception If failed. */ + @Test + public void testStartClusterSnapshotRestoreMultipleThreadsDiffNode() throws Exception { + AtomicInteger nodeIdx = new AtomicInteger(); + + checkStartClusterSnapshotRestoreMultithreaded(nodeIdx::getAndIncrement); + } + + /** + * @param nodeIdxSupplier Ignite node index supplier. + */ + public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSupplier) throws Exception { + startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + CountDownLatch startLatch = new CountDownLatch(1); + AtomicInteger successCnt = new AtomicInteger(); + + IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(() -> { + try { + startLatch.await(TIMEOUT, TimeUnit.MILLISECONDS); + + grid(nodeIdxSupplier.getAsInt()).snapshot().restoreSnapshot( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + + successCnt.incrementAndGet(); + } + catch (Exception ignore) { + // Expected exception. + } + }, 2, "runner"); + + startLatch.countDown(); + + fut.get(TIMEOUT); + + assertEquals(1, successCnt.get()); + } + /** * Ensures that the cache doesn't start if one of the baseline nodes fails. * @@ -144,22 +208,6 @@ public void testCacheStartFailOnNodeLeft() throws Exception { ensureCacheDirEmpty(dfltCacheCfg); } - /** @throws Exception If failed. */ - @Test - public void testBasicClusterSnapshotRestore() throws Exception { - int keysCnt = 10_000; - - IgniteEx ignite = startGridsWithSnapshot(2, keysCnt, true); - - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); - - assertTrue(cache.indexReadyFuture().isDone()); - - checkCacheKeys(cache, keysCnt); - } - /** @throws Exception If failed. */ @Test public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { From 13136de09c28e794eedfaed2a46c5686c26cdecc Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 15 Apr 2021 20:27:09 +0300 Subject: [PATCH 83/98] IGNITE-13805 (minor) Create snapshot test. --- .../IgniteClusterSnapshotRestoreSelfTest.java | 98 +++++++++++-------- 1 file changed, 59 insertions(+), 39 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 e744ea1c8dbab3..4be50981ba1fd4 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 @@ -153,7 +153,7 @@ public void testStartClusterSnapshotRestoreMultipleThreadsDiffNode() throws Exce * @param nodeIdxSupplier Ignite node index supplier. */ public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSupplier) throws Exception { - startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); CountDownLatch startLatch = new CountDownLatch(1); AtomicInteger successCnt = new AtomicInteger(); @@ -177,6 +177,36 @@ public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSup fut.get(TIMEOUT); assertEquals(1, successCnt.get()); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + } + + /** @throws Exception If failed. */ + @Test + public void testCreateSnapshotDuringRestore() throws Exception { + Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + BlockingCustomMessageDiscoverySpi discoSpi = discoSpi(grid(0)); + + discoSpi.block((msg) -> msg instanceof DynamicCacheChangeBatch); + + IgniteFuture fut = + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + discoSpi.waitBlocked(TIMEOUT); + + GridTestUtils.assertThrowsAnyCause( + log, + () -> grid(1).snapshot().createSnapshot("NEW_SNAPSHOT").get(TIMEOUT), + IgniteException.class, + "Cache group restore operation is currently in progress." + ); + + discoSpi.unblock(); + + fut.get(TIMEOUT); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); } /** @@ -485,57 +515,47 @@ private void checkCacheStartWithTheSameName( /** @throws Exception If failed. */ @Test public void testNodeFail() throws Exception { - checkTopologyChange(true); - } - - /** @throws Exception If failed. */ - @Test - public void testNodeJoin() throws Exception { - checkTopologyChange(false); - } - - /** - * @param stopNode {@code True} to check node fail, {@code False} to check node join. - * @throws Exception if failed. - */ - private void checkTopologyChange(boolean stopNode) throws Exception { - int keysCnt = 10_000; - - IgniteEx ignite = startGridsWithSnapshot(4, keysCnt); + startGridsWithSnapshot(4, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, dfltCacheCfg.getName()); - if (stopNode) { - IgniteInternalFuture fut0 = runAsync(() -> stopGrid(3, true)); + IgniteInternalFuture fut0 = runAsync(() -> stopGrid(3, true)); - GridTestUtils.assertThrowsAnyCause( - log, - () -> fut.get(TIMEOUT), - ClusterTopologyCheckedException.class, - "Required node has left the cluster" - ); + GridTestUtils.assertThrowsAnyCause( + log, + () -> fut.get(TIMEOUT), + ClusterTopologyCheckedException.class, + "Required node has left the cluster" + ); - fut0.get(TIMEOUT); + fut0.get(TIMEOUT); - awaitPartitionMapExchange(); + awaitPartitionMapExchange(); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheDirEmpty(dfltCacheCfg); - GridTestUtils.assertThrowsAnyCause( - log, - () -> startGrid(3), - IgniteSpiException.class, - "to add the node to cluster - remove directories with the caches" - ); + GridTestUtils.assertThrowsAnyCause( + log, + () -> startGrid(3), + IgniteSpiException.class, + "to add the node to cluster - remove directories with the caches" + ); + } - return; - } + /** @throws Exception If failed. */ + @Test + public void testNodeJoin() throws Exception { + Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + + IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, dfltCacheCfg.getName()); GridTestUtils.assertThrowsAnyCause( log, - () -> startGrid(4), + () -> startGrid(2), IgniteSpiException.class, "Joining node during caches restore is not allowed" ); @@ -548,7 +568,7 @@ private void checkTopologyChange(boolean stopNode) throws Exception { assertTrue(cache.indexReadyFuture().isDone()); - checkCacheKeys(cache, keysCnt); + checkCacheKeys(cache, CACHE_KEYS_RANGE); } /** From a570502ed7332e085ba4c59ceba257d8e81de022 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 16 Apr 2021 10:08:08 +0300 Subject: [PATCH 84/98] IGNITE-13805 Restore all caches from the snapshot. --- .../org/apache/ignite/IgniteSnapshot.java | 5 ++- .../snapshot/IgniteSnapshotManager.java | 4 +- .../snapshot/SnapshotOperationRequest.java | 5 ++- .../snapshot/SnapshotRestoreProcess.java | 37 +++++++++++-------- .../IgniteClusterSnapshotRestoreSelfTest.java | 32 +++++++++++++++- 5 files changed, 60 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index 4d0f5ce3bf2526..f3e3d4204c97b9 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -19,6 +19,7 @@ import java.util.Collection; import org.apache.ignite.lang.IgniteFuture; +import org.jetbrains.annotations.Nullable; /** * This interface provides functionality for creating cluster-wide cache data snapshots. @@ -54,8 +55,8 @@ public interface IgniteSnapshot { * Restore cache group(s) from the snapshot. * * @param name Snapshot name. - * @param cacheGroupNames Cache groups to be restored. + * @param cacheGroupNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot. * @return Future which will be completed when restore operation finished. */ - public IgniteFuture restoreSnapshot(String name, Collection cacheGroupNames); + public IgniteFuture restoreSnapshot(String name, @Nullable Collection cacheGroupNames); } 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 95403770f44c8c..765e8605f49f1e 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 @@ -1123,10 +1123,10 @@ public List readSnapshotMetadatas(String snpName) { } /** {@inheritDoc} */ - @Override public IgniteFuture restoreSnapshot(String name, Collection grpNames) { + @Override public IgniteFuture restoreSnapshot(String name, @Nullable Collection grpNames) { 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_"); - A.ensure(!F.isEmpty(grpNames), "List of cache group names cannot be empty."); + A.ensure(grpNames == null || !grpNames.isEmpty(), "List of cache group names cannot be empty."); return restoreCacheGrpProc.start(name, grpNames); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java index b8bd5a26a126da..177133fe1f7dc8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotOperationRequest.java @@ -24,6 +24,7 @@ import org.apache.ignite.internal.util.distributed.DistributedProcess; import org.apache.ignite.internal.util.tostring.GridToStringInclude; import org.apache.ignite.internal.util.typedef.internal.S; +import org.jetbrains.annotations.Nullable; /** * Snapshot operation start request for {@link DistributedProcess} initiate message. @@ -63,7 +64,7 @@ public SnapshotOperationRequest( UUID reqId, UUID opNodeId, String snpName, - Collection grps, + @Nullable Collection grps, Set nodes ) { this.reqId = reqId; @@ -90,7 +91,7 @@ public String snapshotName() { /** * @return List of cache group names. */ - public Collection groups() { + public @Nullable Collection groups() { return grps; } 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 aaca8c7a964358..0176e4a5e0947d 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 @@ -145,10 +145,10 @@ protected void cleanup() throws IgniteCheckedException { * Start cache group restore operation. * * @param snpName Snapshot name. - * @param cacheGrpNames Name of the cache groups for restore. + * @param cacheGrpNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot. * @return Future that will be completed when the restore operation is complete and the cache groups are started. */ - public IgniteFuture start(String snpName, Collection cacheGrpNames) { + public IgniteFuture start(String snpName, @Nullable Collection cacheGrpNames) { ClusterSnapshotFuture fut0; try { @@ -190,9 +190,10 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames } Set dataNodes = new HashSet<>(); - Map> metas = f.result(); - Map reqGrpIds = cacheGrpNames.stream().collect(Collectors.toMap(CU::cacheId, v -> v)); Set snpBltNodes = null; + Map> metas = f.result(); + Map reqGrpIds = cacheGrpNames == null ? Collections.emptyMap() : + cacheGrpNames.stream().collect(Collectors.toMap(CU::cacheId, v -> v)); for (Map.Entry> entry : metas.entrySet()) { SnapshotMetadata meta = F.first(entry.getValue()); @@ -210,6 +211,13 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames reqGrpIds.keySet().removeAll(meta.partitions().keySet()); } + if (snpBltNodes == null) { + finishProcess(fut0.rqId, new IllegalArgumentException(OP_REJECT_MSG + "No snapshot data " + + "has been found [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); + + return; + } + if (!reqGrpIds.isEmpty()) { finishProcess(fut0.rqId, new IllegalArgumentException(OP_REJECT_MSG + "Cache group(s) was not " + "found in the snapshot [groups=" + reqGrpIds.values() + ", snapshot=" + snpName + ']')); @@ -220,8 +228,6 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames Collection bltNodes = F.viewReadOnly(ctx.discovery().serverNodes(AffinityTopologyVersion.NONE), node -> node.consistentId().toString(), (node) -> CU.baselineNode(node, ctx.state().clusterState())); - assert !F.isEmpty(snpBltNodes); - snpBltNodes.removeAll(bltNodes); if (!snpBltNodes.isEmpty()) { @@ -251,8 +257,8 @@ public IgniteFuture start(String snpName, Collection cacheGrpNames return; } - SnapshotOperationRequest req = - new SnapshotOperationRequest(fut0.rqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); + SnapshotOperationRequest req = new SnapshotOperationRequest( + fut0.rqId, F.first(dataNodes), snpName, cacheGrpNames, dataNodes); prepareRestoreProc.start(req.requestId(), req); } @@ -426,9 +432,6 @@ private IgniteInternalFuture> prepare(SnapshotOperati } } - for (String grpName : req.groups()) - ensureCacheAbsent(grpName); - opCtx = prepareContext(req); SnapshotRestoreContext opCtx0 = opCtx; @@ -438,13 +441,17 @@ private IgniteInternalFuture> prepare(SnapshotOperati // Ensure that shared cache groups has no conflicts. for (StoredCacheData cfg : opCtx0.cfgs.values()) { + ensureCacheAbsent(cfg.config().getName()); + if (!F.isEmpty(cfg.config().getGroupName())) - ensureCacheAbsent(cfg.config().getName()); + ensureCacheAbsent(cfg.config().getGroupName()); } if (log.isInfoEnabled()) { - log.info("Starting local snapshot restore operation [reqId=" + req.requestId() + - ", snapshot=" + req.snapshotName() + ", group(s)=" + req.groups() + ']'); + log.info("Starting local snapshot restore operation" + + " [reqId=" + req.requestId() + + ", snapshot=" + req.snapshotName() + + ", cache(s)=" + F.viewReadOnly(opCtx0.cfgs.values(), data -> data.config().getName()) + ']'); } Consumer errHnd = (ex) -> opCtx.err.compareAndSet(null, ex); @@ -603,7 +610,7 @@ private SnapshotRestoreContext prepareContext(SnapshotOperationRequest req) thro for (File snpCacheDir : cctx.snapshotMgr().snapshotCacheDirectories(req.snapshotName(), meta.folderName())) { String grpName = FilePageStoreManager.cacheGroupName(snpCacheDir); - if (!req.groups().contains(grpName)) + if (!F.isEmpty(req.groups()) && !req.groups().contains(grpName)) continue; File cacheDir = pageStore.cacheWorkDir(snpCacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX), grpName); 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 4be50981ba1fd4..8f37c7b9543248 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 @@ -135,6 +135,34 @@ public void testBasicClusterSnapshotRestore() throws Exception { checkCacheKeys(cache, keysCnt); } + /** @throws Exception If failed. */ + @Test + public void testRestoreAllGroups() throws Exception { + CacheConfiguration cacheCfg1 = + txCacheConfig(new CacheConfiguration(CACHE1)).setGroupName(SHARED_GRP); + + CacheConfiguration cacheCfg2 = + txCacheConfig(new CacheConfiguration(CACHE2)).setGroupName(SHARED_GRP); + + IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, + dfltCacheCfg.setBackups(0), cacheCfg1, cacheCfg2); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); + + ignite.cache(CACHE1).destroy(); + ignite.cache(CACHE2).destroy(); + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + // Restore all cache groups. + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT); + + checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); + } + /** @throws Exception If failed. */ @Test public void testStartClusterSnapshotRestoreMultipleThreadsSameNode() throws Exception { @@ -514,7 +542,7 @@ private void checkCacheStartWithTheSameName( /** @throws Exception If failed. */ @Test - public void testNodeFail() throws Exception { + public void testNodeFailDuringRestore() throws Exception { startGridsWithSnapshot(4, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); @@ -546,7 +574,7 @@ public void testNodeFail() throws Exception { /** @throws Exception If failed. */ @Test - public void testNodeJoin() throws Exception { + public void testNodeJoinDuringRestore() throws Exception { Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); From bad2d2ce9bc5c0c7fc1c3552d7b866a31ab50a44 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 16 Apr 2021 13:03:12 +0300 Subject: [PATCH 85/98] IGNITE-13805 Improved sql/index validation. --- .../IgniteClusterSnapshotRestoreSelfTest.java | 52 ++++++++++++++----- 1 file changed, 38 insertions(+), 14 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 8f37c7b9543248..ffc1ab00356ade 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 @@ -34,6 +34,7 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteSnapshot; +import org.apache.ignite.binary.BinaryBasicNameMapper; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; @@ -41,6 +42,7 @@ import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.configuration.CacheConfiguration; @@ -53,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; @@ -75,8 +78,8 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe /** Timeout. */ private static final long TIMEOUT = 15_000; - /** Binary type name. */ - private static final String BIN_TYPE_NAME = "customType"; + /** Type name used for binary and SQL. */ + private static final String TYPE_NAME = IndexedObject.class.getName(); /** Cache 1 name. */ private static final String CACHE1 = "cache1"; @@ -101,8 +104,8 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe cfg.setCacheConfiguration(cacheCfgs); else if (dfltCacheCfg != null) { dfltCacheCfg.setSqlIndexMaxInlineSize(255); - dfltCacheCfg.setQueryEntities( - Arrays.asList(queryEntity(BIN_TYPE_NAME), queryEntity(IndexedObject.class.getName()))); + dfltCacheCfg.setSqlSchema("PUBLIC"); + dfltCacheCfg.setQueryEntities(Collections.singletonList(queryEntity(TYPE_NAME))); } return cfg; @@ -116,7 +119,7 @@ private QueryEntity queryEntity(String typeName) { .setKeyType(Integer.class.getName()) .setValueType(typeName) .setFields(new LinkedHashMap<>(F.asMap("id", Integer.class.getName(), "name", String.class.getName()))) - .setIndexes(Arrays.asList(new QueryIndex("id"), new QueryIndex("name"))); + .setIndexes(Collections.singletonList(new QueryIndex("id"))); } /** @throws Exception If failed. */ @@ -271,12 +274,12 @@ public void testCacheStartFailOnNodeLeft() throws Exception { public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { int keysCnt = 10_000; - valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, TYPE_NAME); IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); // Remove metadata. - int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); + int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); ignite.context().cacheObjects().removeType(typeId); @@ -314,7 +317,7 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { int keysCnt = 10_000; - valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, TYPE_NAME); startGridsWithCache(nodesCnt - 2, keysCnt, valBuilder, dfltCacheCfg); @@ -333,7 +336,7 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { awaitPartitionMapExchange(); // Remove metadata. - int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); + int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); ignite.context().cacheObjects().removeType(typeId); @@ -414,18 +417,18 @@ public void testRestoreSharedCacheGroup() throws Exception { /** @throws Exception If failed. */ @Test public void testIncompatibleMetasUpdate() throws Exception { - valBuilder = new BinaryValueBuilder(0, BIN_TYPE_NAME); + valBuilder = new BinaryValueBuilder(0, TYPE_NAME); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); - int typeId = ignite.context().cacheObjects().typeId(BIN_TYPE_NAME); + int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); ignite.context().cacheObjects().removeType(typeId); BinaryObject[] objs = new BinaryObject[CACHE_KEYS_RANGE]; IgniteCache cache1 = createCacheWithBinaryType(ignite, "cache1", n -> { - BinaryObjectBuilder builder = ignite.binary().builder(BIN_TYPE_NAME); + BinaryObjectBuilder builder = ignite.binary().builder(TYPE_NAME); builder.setField("id", n); @@ -455,7 +458,7 @@ public void testIncompatibleMetasUpdate() throws Exception { // Create cache with incompatible binary type. cache1 = createCacheWithBinaryType(ignite, "cache1", n -> { - BinaryObjectBuilder builder = ignite.binary().builder(BIN_TYPE_NAME); + BinaryObjectBuilder builder = ignite.binary().builder(TYPE_NAME); builder.setField("id", UUID.randomUUID()); @@ -785,6 +788,27 @@ private void checkCacheKeys(IgniteCache cache, int keysCnt) { for (int i = 0; i < keysCnt; i++) assertEquals(valBuilder.apply(i), cache.get(i)); + + //noinspection unchecked + if (!grid(0).context().query().moduleEnabled() || + F.isEmpty(cache.getConfiguration(CacheConfiguration.class).getQueryEntities())) + return; + + String tblName = new BinaryBasicNameMapper(true).typeName(TYPE_NAME); + + for (Ignite grid : G.allGrids()) { + GridQueryProcessor qry = ((IgniteEx)grid).context().query(); + + // Make sure SQL works fine. + assertEquals((long)keysCnt, qry.querySqlFields(new SqlFieldsQuery( + "SELECT count(*) FROM " + tblName), true).getAll().get(0).get(0)); + + // Make sure the index is in use. + String explainPlan = (String)qry.querySqlFields(new SqlFieldsQuery( + "explain SELECT * FROM " + tblName + " WHERE id < 10"), true).getAll().get(0).get(0); + + assertTrue("id=" + grid.cluster().localNode().id() + "\n" + explainPlan, explainPlan.contains("ID_ASC_IDX")); + } } /** */ @@ -802,7 +826,7 @@ private static class IndexedObject { private final int id; /** Name. */ - @QuerySqlField(index = true) + @QuerySqlField private final String name; /** From fe167545c126ad8c982908b0a0da51840e49e719 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Mon, 19 Apr 2021 19:54:04 +0300 Subject: [PATCH 86/98] IGNITE-13805 Cache start on stable top only fix. --- .../GridDhtPartitionsExchangeFuture.java | 30 +++----- .../IgniteClusterSnapshotRestoreSelfTest.java | 68 +++++++++++++++++-- 2 files changed, 73 insertions(+), 25 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 6906fadeb5c19c..314e6c68d2d29a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -3827,28 +3827,10 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe return; try { - if (dynamicCacheStartExchange() && isRollbackSupported()) { - if (!F.isEmpty(exchangeGlobalExceptions)) { - sendExchangeFailureMessage(); - - return; - } - - for (UUID nodeId : exchActions.cacheStartRequiredAliveNodes()) { - ClusterNode node = cctx.discovery().node(nodeId); - - if (node != null && - cctx.discovery().alive(node) && - CU.baselineNode(node, cctx.kernalContext().state().clusterState())) - continue; + if (!F.isEmpty(exchangeGlobalExceptions) && dynamicCacheStartExchange() && isRollbackSupported()) { + sendExchangeFailureMessage(); - exchangeGlobalExceptions.put(cctx.localNodeId(), new ClusterTopologyCheckedException( - "Required node has left the cluster [nodeId=" + nodeId + ']')); - - sendExchangeFailureMessage(); - - return; - } + return; } AffinityTopologyVersion resTopVer = exchCtx.events().topologyVersion(); @@ -5144,6 +5126,12 @@ public void onNodeLeft(final ClusterNode node) { if (crd0 == null) finishState = new FinishState(null, initialVersion(), null); + + if (dynamicCacheStartExchange() && + exchActions.cacheStartRequiredAliveNodes().contains(node.id())) { + exchangeGlobalExceptions.put(cctx.localNodeId(), new ClusterTopologyCheckedException( + "Required node has left the cluster [nodeId=" + node.id() + ']')); + } } if (crd0 == null) { 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 ffc1ab00356ade..aee73508db8c49 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 @@ -54,6 +54,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; 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.FilePageStoreManager; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; @@ -246,10 +247,8 @@ public void testCreateSnapshotDuringRestore() throws Exception { * @throws Exception If failed. */ @Test - public void testCacheStartFailOnNodeLeft() throws Exception { - int keysCnt = 10_000; - - startGridsWithSnapshot(3, keysCnt, true); + public void testNodeLeftDuringCacheStartOnExchangeInit() throws Exception { + startGridsWithSnapshot(3, CACHE_KEYS_RANGE, true); BlockingCustomMessageDiscoverySpi discoSpi = discoSpi(grid(0)); @@ -269,6 +268,67 @@ public void testCacheStartFailOnNodeLeft() throws Exception { ensureCacheDirEmpty(dfltCacheCfg); } + /** + * Ensures that the cache is not started if non-coordinator node left during the exchange. + * + * @throws Exception If failed. + */ + @Test + public void testNodeLeftDuringCacheStartOnExchangeFinish() throws Exception { + checkNodeLeftOnExchangeFinish( + false, ClusterTopologyCheckedException.class, "Required node has left the cluster"); + } + + /** + * Ensures that the cache is not started if the coordinator left during the exchange. + * + * @throws Exception If failed. + */ + @Test + public void testCrdLeftDuringCacheStartOnExchangeFinish() throws Exception { + checkNodeLeftOnExchangeFinish( + true, IgniteCheckedException.class, "Operation has been cancelled (node is stopping)"); + } + + /** + * @param crdStop {@code True} to stop coordinator node. + * @param expCls Expected exception class. + * @param expMsg Expected exception message. + * @throws Exception If failed. + */ + private void checkNodeLeftOnExchangeFinish( + boolean crdStop, + Class expCls, + String expMsg + ) throws Exception { + startGridsWithSnapshot(3, CACHE_KEYS_RANGE, true); + + TestRecordingCommunicationSpi node1spi = TestRecordingCommunicationSpi.spi(grid(1)); + TestRecordingCommunicationSpi node2spi = TestRecordingCommunicationSpi.spi(grid(2)); + + node1spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); + node2spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); + + IgniteFuture fut = + grid(1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + node1spi.waitForBlocked(); + node2spi.waitForBlocked(); + + stopGrid(crdStop ? 0 : 2, true); + + node1spi.stopBlock(); + + if (crdStop) + node2spi.stopBlock(); + + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), expCls, expMsg); + + awaitPartitionMapExchange(); + + ensureCacheDirEmpty(dfltCacheCfg); + } + /** @throws Exception If failed. */ @Test public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { From 74ab3cc26e56412f1e9635db6de8a6073f681dd6 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 20 Apr 2021 13:41:48 +0300 Subject: [PATCH 87/98] IGNITE-13805 Test for corrupted files. --- .../snapshot/IgniteSnapshotManager.java | 7 ++ .../snapshot/SnapshotRestoreProcess.java | 8 +- .../IgniteClusterSnapshotRestoreSelfTest.java | 115 ++++++++++++++++++ 3 files changed, 126 insertions(+), 4 deletions(-) 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 765e8605f49f1e..5ff56aadbd36ce 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 @@ -1390,6 +1390,13 @@ void ioFactory(FileIOFactory ioFactory) { this.ioFactory = ioFactory; } + /** + * @return Factory to create IO interface over a page stores. + */ + FileIOFactory ioFactory() { + return ioFactory; + } + /** * @return Relative configured path of persistence data storage directory for the local node. * Example: {@code snapshotWorkDir/db/IgniteNodeName0} 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 0176e4a5e0947d..71059ae0eef8ba 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 @@ -78,12 +78,12 @@ * Distributed process to restore cache group from the snapshot. */ public class SnapshotRestoreProcess { + /** Temporary cache directory prefix. */ + public static final String TMP_CACHE_DIR_PREFIX = ".tmp.snp.restore."; + /** Reject operation message. */ private static final String OP_REJECT_MSG = "Cache group restore operation was rejected. "; - /** Temporary cache directory prefix. */ - private static final String TMP_CACHE_DIR_PREFIX = ".tmp.snp.restore."; - /** Kernal context. */ private final GridKernalContext ctx; @@ -562,7 +562,7 @@ private CompletableFuture restoreAsync( ", target=" + target + "]"); } - Files.copy(snpFile.toPath(), target.toPath()); + IgniteSnapshotManager.copy(snapshotMgr.ioFactory(), snpFile, target, snpFile.length()); } catch (Throwable t) { errHnd.accept(t); 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 aee73508db8c49..4682b129793555 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 @@ -18,6 +18,9 @@ package org.apache.ignite.internal.processors.cache.persistence.snapshot; import java.io.File; +import java.io.IOException; +import java.nio.file.OpenOption; +import java.nio.file.Paths; import java.util.Arrays; import java.util.Collections; import java.util.LinkedHashMap; @@ -26,6 +29,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.IntSupplier; import org.apache.ignite.Ignite; @@ -40,6 +44,7 @@ import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cache.CacheExistsException; +import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.query.SqlFieldsQuery; @@ -50,24 +55,32 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; 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; +import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; +import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.spi.IgniteSpiException; import org.apache.ignite.testframework.GridTestUtils; import org.jetbrains.annotations.Nullable; import org.junit.Test; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess.TMP_CACHE_DIR_PREFIX; 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.runAsync; @@ -97,6 +110,9 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe /** Cache value builder. */ private Function valBuilder = new IndexedValueBuilder(); + /** {@code true} if node should be started in separate jvm. */ + protected volatile boolean jvm; + /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { IgniteConfiguration cfg = super.getConfiguration(name); @@ -635,6 +651,73 @@ public void testNodeFailDuringRestore() throws Exception { ); } + /** @throws Exception If failed. */ + @Test + public void testNodeFailDuringFilesCopy() throws Exception { + dfltCacheCfg.setCacheMode(CacheMode.REPLICATED); + + IgniteEx ignite = startGridsWithSnapshot(3, CACHE_KEYS_RANGE); + + CountDownLatch stopLatch = new CountDownLatch(1); + CountDownLatch failLatch = new CountDownLatch(1); + + File node2dbDir = ((FilePageStoreManager)grid(2).context().cache().context().pageStore()). + cacheWorkDir(dfltCacheCfg).getParentFile(); + + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); + + spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && + ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); + + grid(2).context().cache().context().snapshotMgr().ioFactory( + new CustomFileIOFactory(new RandomAccessFileIOFactory(), + file -> { + String expPath = Paths.get(CACHE_DIR_PREFIX + dfltCacheCfg.getName(), "part-5.bin").toString(); + + if (file.getPath().endsWith(expPath)) { + stopLatch.countDown(); + + try { + U.await(failLatch, TIMEOUT, TimeUnit.MILLISECONDS); + } + catch (IgniteInterruptedCheckedException ignore) { + // No-op. + } + + throw new RuntimeException(new ClusterTopologyCheckedException("Expected")); + } + })); + + IgniteInternalFuture stopFut = runAsync(() -> { + U.await(stopLatch, TIMEOUT, TimeUnit.MILLISECONDS); + + ((IgniteDiscoverySpi)grid(2).configuration().getDiscoverySpi()).simulateNodeFailure(); + + failLatch.countDown(); + + stopGrid(2, true); + + spi.stopBlock(); + + return null; + }); + + IgniteFuture fut = + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + + stopFut.get(TIMEOUT); + + GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, null); + + File[] files = node2dbDir.listFiles(file -> file.getName().startsWith(TMP_CACHE_DIR_PREFIX)); + assertEquals("A temp directory with potentially corrupted files must exist.", 1, files.length); + + startGrid(2); + + files = node2dbDir.listFiles(file -> file.getName().startsWith(TMP_CACHE_DIR_PREFIX)); + assertEquals("A temp directory should be removed at node startup", 0, files.length); + } + /** @throws Exception If failed. */ @Test public void testNodeJoinDuringRestore() throws Exception { @@ -943,4 +1026,36 @@ private class BinaryValueBuilder implements Function { return builder.build(); } } + + /** + * Custom I/O factory to preprocessing created files. + */ + private static class CustomFileIOFactory implements FileIOFactory { + /** Serial version UID. */ + private static final long serialVersionUID = 0L; + + /** Delegate factory. */ + private final FileIOFactory delegate; + + /** Preprocessor for created files. */ + private final Consumer hnd; + + /** + * @param delegate Delegate factory. + * @param hnd Preprocessor for created files. + */ + public CustomFileIOFactory(FileIOFactory delegate, Consumer hnd) { + this.delegate = delegate; + this.hnd = hnd; + } + + /** {@inheritDoc} */ + @Override public FileIO create(File file, OpenOption... modes) throws IOException { + FileIO delegate = this.delegate.create(file, modes); + + hnd.accept(file); + + return delegate; + } + } } From 5b8da3ae0fe2a7406c0dc832eb533392aebcd12d Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 20 Apr 2021 18:40:35 +0300 Subject: [PATCH 88/98] IGNITE-13805 Simplify test. --- .../IgniteClusterSnapshotRestoreSelfTest.java | 42 +++++++------------ 1 file changed, 15 insertions(+), 27 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 4682b129793555..246734eddeb0d3 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 @@ -55,10 +55,8 @@ import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.IgniteInterruptedCheckedException; import org.apache.ignite.internal.TestRecordingCommunicationSpi; import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi; import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; @@ -80,6 +78,8 @@ import org.junit.Test; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.CACHE_DIR_PREFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.FILE_SUFFIX; +import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.PART_FILE_PREFIX; import static org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotRestoreProcess.TMP_CACHE_DIR_PREFIX; 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; @@ -656,54 +656,40 @@ public void testNodeFailDuringRestore() throws Exception { public void testNodeFailDuringFilesCopy() throws Exception { dfltCacheCfg.setCacheMode(CacheMode.REPLICATED); - IgniteEx ignite = startGridsWithSnapshot(3, CACHE_KEYS_RANGE); + startGridsWithSnapshot(3, CACHE_KEYS_RANGE); + TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(2)); CountDownLatch stopLatch = new CountDownLatch(1); - CountDownLatch failLatch = new CountDownLatch(1); - - File node2dbDir = ((FilePageStoreManager)grid(2).context().cache().context().pageStore()). - cacheWorkDir(dfltCacheCfg).getParentFile(); - - TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); + String failingFilePath = Paths.get(CACHE_DIR_PREFIX + dfltCacheCfg.getName(), + PART_FILE_PREFIX + (dfltCacheCfg.getAffinity().partitions() / 2) + FILE_SUFFIX).toString(); + grid(2).context().cache().context().snapshotMgr().ioFactory( new CustomFileIOFactory(new RandomAccessFileIOFactory(), file -> { - String expPath = Paths.get(CACHE_DIR_PREFIX + dfltCacheCfg.getName(), "part-5.bin").toString(); - - if (file.getPath().endsWith(expPath)) { + if (file.getPath().endsWith(failingFilePath)) { stopLatch.countDown(); - try { - U.await(failLatch, TIMEOUT, TimeUnit.MILLISECONDS); - } - catch (IgniteInterruptedCheckedException ignore) { - // No-op. - } - - throw new RuntimeException(new ClusterTopologyCheckedException("Expected")); + throw new RuntimeException("Test exception"); } })); + File node2dbDir = ((FilePageStoreManager)grid(2).context().cache().context().pageStore()). + cacheWorkDir(dfltCacheCfg).getParentFile(); + IgniteInternalFuture stopFut = runAsync(() -> { U.await(stopLatch, TIMEOUT, TimeUnit.MILLISECONDS); - ((IgniteDiscoverySpi)grid(2).configuration().getDiscoverySpi()).simulateNodeFailure(); - - failLatch.countDown(); - stopGrid(2, true); - spi.stopBlock(); - return null; }); IgniteFuture fut = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); stopFut.get(TIMEOUT); @@ -716,6 +702,8 @@ public void testNodeFailDuringFilesCopy() throws Exception { files = node2dbDir.listFiles(file -> file.getName().startsWith(TMP_CACHE_DIR_PREFIX)); assertEquals("A temp directory should be removed at node startup", 0, files.length); + + ensureCacheDirEmpty(dfltCacheCfg); } /** @throws Exception If failed. */ From 31fb3b16e03d94e4f0203a0fefa70c0d6d45012e Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 21 Apr 2021 16:26:51 +0300 Subject: [PATCH 89/98] IGNITE-13805 Split tests into core and indexing. --- .../snapshot/AbstractSnapshotSelfTest.java | 2 +- .../IgniteClusterSnapshotRestoreBaseTest.java | 100 +++++++ .../IgniteClusterSnapshotRestoreSelfTest.java | 277 +----------------- .../IgniteBasicWithPersistenceTestSuite.java | 2 + ...teClusterSnapshotCheckWithIndexesTest.java | 2 +- ...lusterSnapshotRestoreWithIndexingTest.java | 209 +++++++++++++ .../IgnitePdsWithIndexingTestSuite.java | 4 +- 7 files changed, 323 insertions(+), 273 deletions(-) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java create mode 100644 modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 7ad06f448434a7..8528e8056d5528 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -184,7 +184,7 @@ protected void waitForEvents(List evts) throws IgniteInterruptedChecked * @param ccfg Default cache configuration. * @return Cache configuration. */ - protected static CacheConfiguration txCacheConfig(CacheConfiguration ccfg) { + protected CacheConfiguration txCacheConfig(CacheConfiguration ccfg) { return ccfg.setCacheMode(CacheMode.PARTITIONED) .setBackups(2) .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java new file mode 100644 index 00000000000000..9ab5170e8a1729 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java @@ -0,0 +1,100 @@ +/* + * 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.function.Function; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.binary.BinaryObjectBuilder; +import org.apache.ignite.internal.IgniteEx; + +/** + * Snapshot restore test base. + */ +public abstract class IgniteClusterSnapshotRestoreBaseTest extends AbstractSnapshotSelfTest { + /** Timeout. */ + protected static final long TIMEOUT = 15_000; + + /** Cache value builder. */ + protected abstract Function valueBuilder(); + + /** + * @param nodesCnt Nodes count. + * @param keysCnt Number of keys to create. + * @return Ignite coordinator instance. + * @throws Exception if failed. + */ + protected IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { + return startGridsWithSnapshot(nodesCnt, keysCnt, false); + } + + /** + * @param nodesCnt Nodes count. + * @param keysCnt Number of keys to create. + * @param startClient {@code True} to start an additional client node. + * @return Ignite coordinator instance. + * @throws Exception if failed. + */ + protected IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt, boolean startClient) throws Exception { + IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder(), dfltCacheCfg.setBackups(0)); + + if (startClient) + ignite = startClientGrid("client"); + + ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); + + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + return ignite; + } + + /** + * @param cache Cache. + * @param keysCnt Expected number of keys. + */ + protected void checkCacheKeys(IgniteCache cache, int keysCnt) { + assertEquals(keysCnt, cache.size()); + + for (int i = 0; i < keysCnt; i++) + assertEquals(valueBuilder().apply(i), cache.get(i)); + } + + /** */ + protected class BinaryValueBuilder implements Function { + /** Binary type name. */ + private final String typeName; + + /** + * @param typeName Binary type name. + */ + BinaryValueBuilder(String typeName) { + this.typeName = typeName; + } + + /** {@inheritDoc} */ + @Override public Object apply(Integer key) { + BinaryObjectBuilder builder = grid(0).binary().builder(typeName); + + builder.setField("id", key); + builder.setField("name", String.valueOf(key)); + + return builder.build(); + } + } +} 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 246734eddeb0d3..5ccc79a59b6b55 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 @@ -23,8 +23,6 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.Objects; import java.util.UUID; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -38,17 +36,12 @@ import org.apache.ignite.IgniteException; import org.apache.ignite.IgniteIllegalStateException; import org.apache.ignite.IgniteSnapshot; -import org.apache.ignite.binary.BinaryBasicNameMapper; import org.apache.ignite.binary.BinaryObject; import org.apache.ignite.binary.BinaryObjectBuilder; import org.apache.ignite.binary.BinaryObjectException; import org.apache.ignite.binary.BinaryType; import org.apache.ignite.cache.CacheExistsException; import org.apache.ignite.cache.CacheMode; -import org.apache.ignite.cache.QueryEntity; -import org.apache.ignite.cache.QueryIndex; -import org.apache.ignite.cache.query.SqlFieldsQuery; -import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; @@ -64,10 +57,8 @@ import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory; import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager; import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIOFactory; -import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.distributed.DistributedProcess.DistributedProcessType; import org.apache.ignite.internal.util.distributed.SingleNodeMessage; -import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; @@ -88,12 +79,9 @@ /** * Snapshot restore tests. */ -public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTest { - /** Timeout. */ - private static final long TIMEOUT = 15_000; - +public class IgniteClusterSnapshotRestoreSelfTest extends IgniteClusterSnapshotRestoreBaseTest { /** Type name used for binary and SQL. */ - private static final String TYPE_NAME = IndexedObject.class.getName(); + private static final String TYPE_NAME = "CustomType"; /** Cache 1 name. */ private static final String CACHE1 = "cache1"; @@ -108,10 +96,7 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe private CacheConfiguration[] cacheCfgs; /** Cache value builder. */ - private Function valBuilder = new IndexedValueBuilder(); - - /** {@code true} if node should be started in separate jvm. */ - protected volatile boolean jvm; + private Function valBuilder = String::valueOf; /** {@inheritDoc} */ @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { @@ -119,40 +104,13 @@ public class IgniteClusterSnapshotRestoreSelfTest extends AbstractSnapshotSelfTe if (cacheCfgs != null) cfg.setCacheConfiguration(cacheCfgs); - else if (dfltCacheCfg != null) { - dfltCacheCfg.setSqlIndexMaxInlineSize(255); - dfltCacheCfg.setSqlSchema("PUBLIC"); - dfltCacheCfg.setQueryEntities(Collections.singletonList(queryEntity(TYPE_NAME))); - } return cfg; } - /** - * @param typeName Type name. - */ - private QueryEntity queryEntity(String typeName) { - return new QueryEntity() - .setKeyType(Integer.class.getName()) - .setValueType(typeName) - .setFields(new LinkedHashMap<>(F.asMap("id", Integer.class.getName(), "name", String.class.getName()))) - .setIndexes(Collections.singletonList(new QueryIndex("id"))); - } - - /** @throws Exception If failed. */ - @Test - public void testBasicClusterSnapshotRestore() throws Exception { - int keysCnt = 10_000; - - IgniteEx client = startGridsWithSnapshot(2, keysCnt, true); - - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - - IgniteCache cache = client.cache(dfltCacheCfg.getName()); - - assertTrue(cache.indexReadyFuture().isDone()); - - checkCacheKeys(cache, keysCnt); + /** {@inheritDoc} */ + @Override protected Function valueBuilder() { + return valBuilder; } /** @throws Exception If failed. */ @@ -345,31 +303,6 @@ private void checkNodeLeftOnExchangeFinish( ensureCacheDirEmpty(dfltCacheCfg); } - /** @throws Exception If failed. */ - @Test - public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { - int keysCnt = 10_000; - - valBuilder = new BinaryValueBuilder(0, TYPE_NAME); - - IgniteEx ignite = startGridsWithSnapshot(2, keysCnt); - - // Remove metadata. - int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); - - ignite.context().cacheObjects().removeType(typeId); - - forceCheckpoint(); - - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); - - assertTrue(cache.indexReadyFuture().isDone()); - - checkCacheKeys(cache, keysCnt); - } - /** @throws Exception If failed. */ @Test public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception { @@ -386,57 +319,10 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); } - /** @throws Exception If failed. */ - @Test - public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { - int nodesCnt = 4; - - int keysCnt = 10_000; - - valBuilder = new BinaryValueBuilder(0, TYPE_NAME); - - startGridsWithCache(nodesCnt - 2, keysCnt, valBuilder, dfltCacheCfg); - - grid(0).snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - - startGrid(nodesCnt - 2); - - IgniteEx ignite = startGrid(nodesCnt - 1); - - resetBaselineTopology(); - - awaitPartitionMapExchange(); - - ignite.cache(dfltCacheCfg.getName()).destroy(); - - awaitPartitionMapExchange(); - - // Remove metadata. - int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); - - ignite.context().cacheObjects().removeType(typeId); - - forceCheckpoint(); - - // Restore from an empty node. - ignite.snapshot().restoreSnapshot( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); - - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); - - assertTrue(cache.indexReadyFuture().isDone()); - - awaitPartitionMapExchange(); - - checkCacheKeys(cache, keysCnt); - } - /** @throws Exception If failed. */ @Test public void testClusterSnapshotRestoreOnSmallerTopology() throws Exception { - int keysCnt = 10_000; - - startGridsWithSnapshot(2, keysCnt, true); + startGridsWithSnapshot(2, CACHE_KEYS_RANGE, true); stopGrid(1); @@ -493,7 +379,7 @@ public void testRestoreSharedCacheGroup() throws Exception { /** @throws Exception If failed. */ @Test public void testIncompatibleMetasUpdate() throws Exception { - valBuilder = new BinaryValueBuilder(0, TYPE_NAME); + valBuilder = new BinaryValueBuilder(TYPE_NAME); IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); @@ -845,38 +731,6 @@ private void ensureCacheDirEmpty(CacheConfiguration ccfg) throws IgniteChe } } - /** - * @param nodesCnt Nodes count. - * @param keysCnt Number of keys to create. - * @return Ignite coordinator instance. - * @throws Exception if failed. - */ - private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exception { - return startGridsWithSnapshot(nodesCnt, keysCnt, false); - } - - /** - * @param nodesCnt Nodes count. - * @param keysCnt Number of keys to create. - * @param startClient {@code True} to start an additional client node. - * @return Ignite coordinator instance. - * @throws Exception if failed. - */ - private IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt, boolean startClient) throws Exception { - IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valBuilder, dfltCacheCfg.setBackups(0)); - - if (startClient) - ignite = startClientGrid("client"); - - ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); - - ignite.cache(dfltCacheCfg.getName()).destroy(); - - awaitPartitionMapExchange(); - - return ignite; - } - /** * @param spi Test communication spi. * @param restorePhase The type of distributed process on which communication is blocked. @@ -900,121 +754,6 @@ private IgniteFuture waitForBlockOnRestore( return fut; } - /** - * @param cache Cache. - * @param startIdx The initial value of the number for the key. - * @param cnt Number of entries to put in the cache. - */ - private void putKeys(IgniteCache cache, int startIdx, int cnt) { - for (int i = startIdx; i < (startIdx + cnt); i++) - cache.put(i, valBuilder.apply(i)); - } - - /** - * @param cache Cache. - * @param keysCnt Expected number of keys. - */ - private void checkCacheKeys(IgniteCache cache, int keysCnt) { - assertEquals(keysCnt, cache.size()); - - for (int i = 0; i < keysCnt; i++) - assertEquals(valBuilder.apply(i), cache.get(i)); - - //noinspection unchecked - if (!grid(0).context().query().moduleEnabled() || - F.isEmpty(cache.getConfiguration(CacheConfiguration.class).getQueryEntities())) - return; - - String tblName = new BinaryBasicNameMapper(true).typeName(TYPE_NAME); - - for (Ignite grid : G.allGrids()) { - GridQueryProcessor qry = ((IgniteEx)grid).context().query(); - - // Make sure SQL works fine. - assertEquals((long)keysCnt, qry.querySqlFields(new SqlFieldsQuery( - "SELECT count(*) FROM " + tblName), true).getAll().get(0).get(0)); - - // Make sure the index is in use. - String explainPlan = (String)qry.querySqlFields(new SqlFieldsQuery( - "explain SELECT * FROM " + tblName + " WHERE id < 10"), true).getAll().get(0).get(0); - - assertTrue("id=" + grid.cluster().localNode().id() + "\n" + explainPlan, explainPlan.contains("ID_ASC_IDX")); - } - } - - /** */ - private static class IndexedValueBuilder implements Function { - /** {@inheritDoc} */ - @Override public Object apply(Integer key) { - return new IndexedObject(key, "Person number #" + key); - } - } - - /** */ - private static class IndexedObject { - /** Id. */ - @QuerySqlField(index = true) - private final int id; - - /** Name. */ - @QuerySqlField - private final String name; - - /** - * @param id Id. - */ - public IndexedObject(int id, String name) { - this.id = id; - this.name = name; - } - - /** {@inheritDoc} */ - @Override public boolean equals(Object o) { - if (this == o) - return true; - - if (o == null || getClass() != o.getClass()) - return false; - - IndexedObject obj = (IndexedObject)o; - - return id == obj.id && Objects.equals(name, obj.name); - } - - /** {@inheritDoc} */ - @Override public int hashCode() { - return Objects.hash(name, id); - } - } - - /** */ - private class BinaryValueBuilder implements Function { - /** Ignite node index. */ - private final int nodeIdx; - - /** Binary type name. */ - private final String typeName; - - /** - * @param nodeIdx Ignite node index. - * @param typeName Binary type name. - */ - BinaryValueBuilder(int nodeIdx, String typeName) { - this.nodeIdx = nodeIdx; - this.typeName = typeName; - } - - /** {@inheritDoc} */ - @Override public Object apply(Integer key) { - BinaryObjectBuilder builder = grid(nodeIdx).binary().builder(typeName); - - builder.setField("id", key); - builder.setField("name", String.valueOf(key)); - - return builder.build(); - } - } - /** * Custom I/O factory to preprocessing created files. */ diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java index 14cd9d74a1845b..fbbe62ec0c6cbd 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicWithPersistenceTestSuite.java @@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.persistence.CommonPoolStarvationCheckpointTest; import org.apache.ignite.internal.processors.cache.persistence.SingleNodePersistenceSslTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotCheckTest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotRestoreSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotSelfTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotMXBeanTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManagerSelfTest; @@ -99,6 +100,7 @@ IgniteClusterSnapshotSelfTest.class, IgniteClusterSnapshotCheckTest.class, IgniteSnapshotMXBeanTest.class, + IgniteClusterSnapshotRestoreSelfTest.class, IgniteClusterIdTagTest.class, FullyConnectedComponentSearcherTest.class, 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 e0aee42e3270a9..7f4165d243e596 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 @@ -119,7 +119,7 @@ public SelfNodeFilter(UUID nodeId) { * @param cacheName Cache name. * @return Cache configuration. */ - private static CacheConfiguration txFilteredCache(String cacheName) { + private CacheConfiguration txFilteredCache(String cacheName) { return txCacheConfig(new CacheConfiguration(cacheName)) .setCacheMode(CacheMode.REPLICATED) .setQueryEntities(singletonList(new QueryEntity(Integer.class.getName(), Account.class.getName()))); diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java new file mode 100644 index 00000000000000..bdde50a5237522 --- /dev/null +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java @@ -0,0 +1,209 @@ +/* + * 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.Collections; +import java.util.LinkedHashMap; +import java.util.Objects; +import java.util.function.Function; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.binary.BinaryBasicNameMapper; +import org.apache.ignite.cache.QueryEntity; +import org.apache.ignite.cache.QueryIndex; +import org.apache.ignite.cache.query.SqlFieldsQuery; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.query.GridQueryProcessor; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +import org.junit.Test; + +/** + * Cluster snapshot restore tests verifying SQL and indexing. + */ +public class IgniteClusterSnapshotRestoreWithIndexingTest extends IgniteClusterSnapshotRestoreBaseTest { + /** Type name used for binary and SQL. */ + private static final String TYPE_NAME = IndexedObject.class.getName(); + + /** Number of cache keys to pre-create at node start. */ + private static final int CACHE_KEYS_RANGE = 10_000; + + /** Cache value builder. */ + private Function valBuilder = new BinaryValueBuilder(TYPE_NAME); + + /** {@inheritDoc} */ + @Override protected CacheConfiguration txCacheConfig(CacheConfiguration ccfg) { + return super.txCacheConfig(ccfg).setSqlIndexMaxInlineSize(255).setSqlSchema("PUBLIC") + .setQueryEntities(Collections.singletonList(new QueryEntity() + .setKeyType(Integer.class.getName()) + .setValueType(TYPE_NAME) + .setFields(new LinkedHashMap<>(F.asMap("id", Integer.class.getName(), "name", String.class.getName()))) + .setIndexes(Collections.singletonList(new QueryIndex("id"))))); + } + + /** {@inheritDoc} */ + @Override protected Function valueBuilder() { + return valBuilder; + } + + /** @throws Exception If failed. */ + @Test + public void testBasicClusterSnapshotRestore() throws Exception { + valBuilder = new IndexedValueBuilder(); + + IgniteEx client = startGridsWithSnapshot(2, CACHE_KEYS_RANGE, true); + + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + + IgniteCache cache = client.cache(dfltCacheCfg.getName()); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, CACHE_KEYS_RANGE); + } + + /** @throws Exception If failed. */ + @Test + public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { + IgniteEx ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); + + // Remove metadata. + int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); + + ignite.context().cacheObjects().removeType(typeId); + + forceCheckpoint(); + + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); + + assertTrue(cache.indexReadyFuture().isDone()); + + checkCacheKeys(cache, CACHE_KEYS_RANGE); + } + + /** @throws Exception If failed. */ + @Test + public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { + int nodesCnt = 4; + + startGridsWithCache(nodesCnt - 2, CACHE_KEYS_RANGE, valBuilder, dfltCacheCfg); + + grid(0).snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); + + startGrid(nodesCnt - 2); + + IgniteEx ignite = startGrid(nodesCnt - 1); + + resetBaselineTopology(); + + awaitPartitionMapExchange(); + + ignite.cache(dfltCacheCfg.getName()).destroy(); + + awaitPartitionMapExchange(); + + // Remove metadata. + int typeId = ignite.context().cacheObjects().typeId(TYPE_NAME); + + ignite.context().cacheObjects().removeType(typeId); + + forceCheckpoint(); + + // Restore from an empty node. + ignite.snapshot().restoreSnapshot( + SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + + IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); + + assertTrue(cache.indexReadyFuture().isDone()); + + awaitPartitionMapExchange(); + + checkCacheKeys(cache, CACHE_KEYS_RANGE); + } + + /** {@inheritDoc} */ + @Override protected void checkCacheKeys(IgniteCache cache, int keysCnt) { + super.checkCacheKeys(cache, keysCnt); + + String tblName = new BinaryBasicNameMapper(true).typeName(TYPE_NAME); + + for (Ignite grid : G.allGrids()) { + GridQueryProcessor qry = ((IgniteEx)grid).context().query(); + + // Make sure SQL works fine. + assertEquals((long)keysCnt, qry.querySqlFields(new SqlFieldsQuery( + "SELECT count(*) FROM " + tblName), true).getAll().get(0).get(0)); + + // Make sure the index is in use. + String explainPlan = (String)qry.querySqlFields(new SqlFieldsQuery( + "explain SELECT * FROM " + tblName + " WHERE id < 10"), true).getAll().get(0).get(0); + + assertTrue("id=" + grid.cluster().localNode().id() + "\n" + explainPlan, explainPlan.contains("ID_ASC_IDX")); + } + } + + /** */ + private static class IndexedValueBuilder implements Function { + /** {@inheritDoc} */ + @Override public Object apply(Integer key) { + return new IndexedObject(key, "Person number #" + key); + } + } + + /** */ + private static class IndexedObject { + /** Id. */ + @QuerySqlField(index = true) + private final int id; + + /** Name. */ + @QuerySqlField + private final String name; + + /** + * @param id Id. + */ + public IndexedObject(int id, String name) { + this.id = id; + this.name = name; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + IndexedObject obj = (IndexedObject)o; + + return id == obj.id && Objects.equals(name, obj.name); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return Objects.hash(name, id); + } + } +} diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java index 9e5d299200cb9b..69cbb71e83b1cb 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgnitePdsWithIndexingTestSuite.java @@ -29,7 +29,7 @@ import org.apache.ignite.internal.processors.cache.persistence.db.LongDestroyDurableBackgroundTaskTest; import org.apache.ignite.internal.processors.cache.persistence.db.MultipleParallelCacheDeleteDeadlockTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotCheckWithIndexesTest; -import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotRestoreSelfTest; +import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotRestoreWithIndexingTest; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteClusterSnapshotWithIndexesTest; import org.apache.ignite.internal.processors.database.IgniteDbMultiNodeWithIndexingPutGetTest; import org.apache.ignite.internal.processors.database.IgniteDbSingleNodeWithIndexingPutGetTest; @@ -68,7 +68,7 @@ IgnitePdsIndexingDefragmentationTest.class, StopRebuildIndexTest.class, ForceRebuildIndexTest.class, - IgniteClusterSnapshotRestoreSelfTest.class + IgniteClusterSnapshotRestoreWithIndexingTest.class }) public class IgnitePdsWithIndexingTestSuite { } From 0eab67c4fffc6f98d12d080126a9644b8ea63320 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 23 Apr 2021 16:18:16 +0300 Subject: [PATCH 90/98] IGNITE-13805 (minor) Snapshot test fix. --- .../snapshot/AbstractSnapshotSelfTest.java | 3 + .../IgniteClusterSnapshotRestoreSelfTest.java | 78 ++++++++----------- ...lusterSnapshotRestoreWithIndexingTest.java | 14 ++-- 3 files changed, 42 insertions(+), 53 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java index 8528e8056d5528..b873f7d0217dcf 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotSelfTest.java @@ -124,6 +124,9 @@ public abstract class AbstractSnapshotSelfTest extends GridCommonAbstractTest { discoSpi.setIpFinder(((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder()); + if (dfltCacheCfg != null) + cfg.setCacheConfiguration(dfltCacheCfg); + return cfg.setConsistentId(igniteInstanceName) .setCommunicationSpi(new TestRecordingCommunicationSpi()) .setDataStorageConfiguration(new DataStorageConfiguration() 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 5ccc79a59b6b55..16ab9f9f4d88d3 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 @@ -44,7 +44,6 @@ import org.apache.ignite.cache.CacheMode; import org.apache.ignite.cluster.ClusterState; import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgniteInternalFuture; @@ -92,22 +91,9 @@ public class IgniteClusterSnapshotRestoreSelfTest extends IgniteClusterSnapshotR /** Default shared cache group name. */ private static final String SHARED_GRP = "shared"; - /** Static cache configurations. */ - private CacheConfiguration[] cacheCfgs; - /** Cache value builder. */ private Function valBuilder = String::valueOf; - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String name) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(name); - - if (cacheCfgs != null) - cfg.setCacheConfiguration(cacheCfgs); - - return cfg; - } - /** {@inheritDoc} */ @Override protected Function valueBuilder() { return valBuilder; @@ -129,14 +115,14 @@ public void testRestoreAllGroups() throws Exception { ignite.cache(CACHE1).destroy(); ignite.cache(CACHE2).destroy(); - ignite.cache(dfltCacheCfg.getName()).destroy(); + ignite.cache(DEFAULT_CACHE_NAME).destroy(); awaitPartitionMapExchange(); // Restore all cache groups. grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT); - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); checkCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); checkCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); } @@ -169,7 +155,7 @@ public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSup startLatch.await(TIMEOUT, TimeUnit.MILLISECONDS); grid(nodeIdxSupplier.getAsInt()).snapshot().restoreSnapshot( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); successCnt.incrementAndGet(); } @@ -184,7 +170,7 @@ public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSup assertEquals(1, successCnt.get()); - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -197,7 +183,7 @@ public void testCreateSnapshotDuringRestore() throws Exception { discoSpi.block((msg) -> msg instanceof DynamicCacheChangeBatch); IgniteFuture fut = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); discoSpi.waitBlocked(TIMEOUT); @@ -212,7 +198,7 @@ public void testCreateSnapshotDuringRestore() throws Exception { fut.get(TIMEOUT); - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); } /** @@ -229,7 +215,7 @@ public void testNodeLeftDuringCacheStartOnExchangeInit() throws Exception { discoSpi.block((msg) -> msg instanceof DynamicCacheChangeBatch); IgniteFuture fut = - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); discoSpi.waitBlocked(TIMEOUT); @@ -239,7 +225,7 @@ public void testNodeLeftDuringCacheStartOnExchangeInit() throws Exception { GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), ClusterTopologyCheckedException.class, null); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); } /** @@ -284,7 +270,7 @@ private void checkNodeLeftOnExchangeFinish( node2spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); IgniteFuture fut = - grid(1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); node1spi.waitForBlocked(); node2spi.waitForBlocked(); @@ -300,7 +286,7 @@ private void checkNodeLeftOnExchangeFinish( awaitPartitionMapExchange(); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); } /** @throws Exception If failed. */ @@ -313,7 +299,7 @@ public void testClusterSnapshotRestoreRejectOnInActiveCluster() throws Exception ignite.cluster().state(ClusterState.INACTIVE); IgniteFuture fut = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); GridTestUtils.assertThrowsAnyCause( log, () -> fut.get(TIMEOUT), IgniteException.class, "The cluster should be active"); @@ -329,11 +315,11 @@ public void testClusterSnapshotRestoreOnSmallerTopology() throws Exception { resetBaselineTopology(); IgniteFuture fut = - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); GridTestUtils.assertThrowsAnyCause(log, () -> fut.get(TIMEOUT), IgniteIllegalStateException.class, null); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); } /** @throws Exception If failed. */ @@ -347,8 +333,6 @@ public void testRestoreSharedCacheGroup() throws Exception { IgniteEx ignite = startGridsWithCache(2, CACHE_KEYS_RANGE, valBuilder, cacheCfg1, cacheCfg2); - cacheCfgs = new CacheConfiguration[] {cacheCfg1, cacheCfg2}; - ignite.cluster().state(ClusterState.ACTIVE); ignite.snapshot().createSnapshot(SNAPSHOT_NAME).get(TIMEOUT); @@ -400,7 +384,7 @@ public void testIncompatibleMetasUpdate() throws Exception { }); IgniteFuture fut = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); fut.get(TIMEOUT); @@ -414,7 +398,7 @@ public void testIncompatibleMetasUpdate() throws Exception { cache1.destroy(); - grid(0).cache(dfltCacheCfg.getName()).destroy(); + grid(0).cache(DEFAULT_CACHE_NAME).destroy(); ignite.context().cacheObjects().removeType(typeId); @@ -430,11 +414,11 @@ public void testIncompatibleMetasUpdate() throws Exception { }); IgniteFuture fut0 = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); GridTestUtils.assertThrowsAnyCause(log, () -> fut0.get(TIMEOUT), BinaryObjectException.class, null); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); for (int i = 0; i < CACHE_KEYS_RANGE; i++) assertEquals(objs[i], cache1.get(i)); @@ -512,7 +496,7 @@ public void testNodeFailDuringRestore() throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(3)); - IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, DEFAULT_CACHE_NAME); IgniteInternalFuture fut0 = runAsync(() -> stopGrid(3, true)); @@ -527,7 +511,7 @@ public void testNodeFailDuringRestore() throws Exception { awaitPartitionMapExchange(); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); GridTestUtils.assertThrowsAnyCause( log, @@ -550,7 +534,7 @@ public void testNodeFailDuringFilesCopy() throws Exception { spi.blockMessages((node, msg) -> msg instanceof SingleNodeMessage && ((SingleNodeMessage)msg).type() == RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE.ordinal()); - String failingFilePath = Paths.get(CACHE_DIR_PREFIX + dfltCacheCfg.getName(), + String failingFilePath = Paths.get(CACHE_DIR_PREFIX + DEFAULT_CACHE_NAME, PART_FILE_PREFIX + (dfltCacheCfg.getAffinity().partitions() / 2) + FILE_SUFFIX).toString(); grid(2).context().cache().context().snapshotMgr().ioFactory( @@ -575,7 +559,7 @@ public void testNodeFailDuringFilesCopy() throws Exception { }); IgniteFuture fut = - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); stopFut.get(TIMEOUT); @@ -584,12 +568,14 @@ public void testNodeFailDuringFilesCopy() throws Exception { File[] files = node2dbDir.listFiles(file -> file.getName().startsWith(TMP_CACHE_DIR_PREFIX)); assertEquals("A temp directory with potentially corrupted files must exist.", 1, files.length); + ensureCacheAbsent(dfltCacheCfg); + + dfltCacheCfg = null; + startGrid(2); files = node2dbDir.listFiles(file -> file.getName().startsWith(TMP_CACHE_DIR_PREFIX)); assertEquals("A temp directory should be removed at node startup", 0, files.length); - - ensureCacheDirEmpty(dfltCacheCfg); } /** @throws Exception If failed. */ @@ -599,7 +585,7 @@ public void testNodeJoinDuringRestore() throws Exception { TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(1)); - IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, RESTORE_CACHE_GROUP_SNAPSHOT_PREPARE, DEFAULT_CACHE_NAME); GridTestUtils.assertThrowsAnyCause( log, @@ -612,7 +598,7 @@ public void testNodeJoinDuringRestore() throws Exception { fut.get(TIMEOUT); - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()); + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME); assertTrue(cache.indexReadyFuture().isDone()); @@ -672,7 +658,7 @@ private void checkClusterStateChange( TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(grid(nodesCnt - 1)); - IgniteFuture fut = waitForBlockOnRestore(spi, procType, dfltCacheCfg.getName()); + IgniteFuture fut = waitForBlockOnRestore(spi, procType, DEFAULT_CACHE_NAME); ignite.cluster().state(state); @@ -683,7 +669,7 @@ private void checkClusterStateChange( ignite.cluster().state(ClusterState.ACTIVE); - checkCacheKeys(ignite.cache(dfltCacheCfg.getName()), CACHE_KEYS_RANGE); + checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); return; } @@ -692,9 +678,9 @@ private void checkClusterStateChange( ignite.cluster().state(ClusterState.ACTIVE); - ensureCacheDirEmpty(dfltCacheCfg); + ensureCacheAbsent(dfltCacheCfg); - String cacheName = dfltCacheCfg.getName(); + String cacheName = DEFAULT_CACHE_NAME; grid(nodesCnt - 1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); @@ -705,7 +691,7 @@ private void checkClusterStateChange( * @param ccfg Cache configuration. * @throws IgniteCheckedException if failed. */ - private void ensureCacheDirEmpty(CacheConfiguration ccfg) throws IgniteCheckedException { + private void ensureCacheAbsent(CacheConfiguration ccfg) throws IgniteCheckedException { String cacheName = ccfg.getName(); for (Ignite ignite : G.allGrids()) { diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java index bdde50a5237522..ff58afc4121970 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java @@ -70,9 +70,9 @@ public void testBasicClusterSnapshotRestore() throws Exception { IgniteEx client = startGridsWithSnapshot(2, CACHE_KEYS_RANGE, true); - grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = client.cache(dfltCacheCfg.getName()); + IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); assertTrue(cache.indexReadyFuture().isDone()); @@ -91,9 +91,9 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { forceCheckpoint(); - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(); assertTrue(cache.indexReadyFuture().isDone()); @@ -117,7 +117,7 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { awaitPartitionMapExchange(); - ignite.cache(dfltCacheCfg.getName()).destroy(); + ignite.cache(DEFAULT_CACHE_NAME).destroy(); awaitPartitionMapExchange(); @@ -130,9 +130,9 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { // Restore from an empty node. ignite.snapshot().restoreSnapshot( - SNAPSHOT_NAME, Collections.singleton(dfltCacheCfg.getName())).get(TIMEOUT); + SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = ignite.cache(dfltCacheCfg.getName()).withKeepBinary(); + IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(); assertTrue(cache.indexReadyFuture().isDone()); From 42fa5e123045681ae211df19e9d8369fb2780e3c Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Tue, 27 Apr 2021 17:05:50 +0300 Subject: [PATCH 91/98] IGNITE-13805 (minor) Code cleanup. --- .../snapshot/SnapshotRestoreProcess.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) 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 71059ae0eef8ba..7ee46826066d53 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 @@ -177,7 +177,8 @@ public IgniteFuture start(String snpName, @Nullable Collection cac fut0 = fut; } - } catch (IgniteException e) { + } + catch (IgniteException e) { return new IgniteFinishedFutureImpl<>(e); } @@ -473,7 +474,8 @@ private IgniteInternalFuture> prepare(SnapshotOperati for (File src : opCtx0.dirs) Files.move(formatTmpDirName(src).toPath(), src.toPath(), StandardCopyOption.ATOMIC_MOVE); - } catch (Throwable t) { + } + catch (Throwable t) { log.error("Unable to restore cache group(s) from the snapshot " + "[reqId=" + opCtx.reqId + ", snapshot=" + opCtx.snpName + ']', t); @@ -486,7 +488,8 @@ private IgniteInternalFuture> prepare(SnapshotOperati }); return retFut; - } catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { + } + catch (IgniteIllegalStateException | IgniteCheckedException | RejectedExecutionException e) { log.error("Unable to restore cache group(s) from the snapshot " + "[reqId=" + req.requestId() + ", snapshot=" + req.snapshotName() + ']', e); @@ -616,6 +619,11 @@ private SnapshotRestoreContext prepareContext(SnapshotOperationRequest req) thro File cacheDir = pageStore.cacheWorkDir(snpCacheDir.getName().startsWith(CACHE_GRP_DIR_PREFIX), grpName); if (cacheDir.exists()) { + if (!cacheDir.isDirectory()) { + throw new IgniteCheckedException("Unable to restore cache group, file with required directory " + + "name already exists [group=" + grpName + ", file=" + cacheDir + ']'); + } + if (cacheDir.list().length > 0) { throw new IgniteCheckedException("Unable to restore cache group, directory is not empty " + "[group=" + grpName + ", dir=" + cacheDir + ']'); From c6b6add96172eddd90fab18d5db39b40202ff72f Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Wed, 28 Apr 2021 22:02:58 +0300 Subject: [PATCH 92/98] IGNITE-13805 Set exchangeLocE on cache start fail. --- .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 314e6c68d2d29a..7e3f465d473f97 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -299,7 +299,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte private boolean forceAffReassignment; /** Exception that was thrown during init phase on local node. */ - private Exception exchangeLocE; + private volatile Exception exchangeLocE; /** Exchange exceptions from all participating nodes. */ private final Map exchangeGlobalExceptions = new ConcurrentHashMap<>(); @@ -5127,9 +5127,9 @@ public void onNodeLeft(final ClusterNode node) { if (crd0 == null) finishState = new FinishState(null, initialVersion(), null); - if (dynamicCacheStartExchange() && + if (dynamicCacheStartExchange() && exchangeLocE == null && exchActions.cacheStartRequiredAliveNodes().contains(node.id())) { - exchangeGlobalExceptions.put(cctx.localNodeId(), new ClusterTopologyCheckedException( + exchangeGlobalExceptions.put(cctx.localNodeId(), exchangeLocE = new ClusterTopologyCheckedException( "Required node has left the cluster [nodeId=" + node.id() + ']')); } } From 32ea6e328b9f88f597a5b05a3682773569075689 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 29 Apr 2021 14:48:54 +0300 Subject: [PATCH 93/98] IGNITE-13805 Review notes. --- .../org/apache/ignite/IgniteSnapshot.java | 2 + .../snapshot/SnapshotRestoreProcess.java | 2 +- .../IgniteClusterSnapshotRestoreBaseTest.java | 4 +- .../IgniteClusterSnapshotRestoreSelfTest.java | 40 +++++++++---------- ...lusterSnapshotRestoreWithIndexingTest.java | 13 +++--- 5 files changed, 31 insertions(+), 30 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index f3e3d4204c97b9..466ed24a7d12d2 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -53,6 +53,8 @@ public interface IgniteSnapshot { /** * Restore cache group(s) from the snapshot. + *

+ * NOTE: Cache groups to be restored from the snapshot must not present in the cluster, if they present, they must be destroyed by the user before starting this operation. * * @param name Snapshot name. * @param cacheGroupNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot. 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 7ee46826066d53..1398109beb7999 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 @@ -79,7 +79,7 @@ */ public class SnapshotRestoreProcess { /** Temporary cache directory prefix. */ - public static final String TMP_CACHE_DIR_PREFIX = ".tmp.snp.restore."; + public static final String TMP_CACHE_DIR_PREFIX = "_tmp_snp_restore_"; /** Reject operation message. */ private static final String OP_REJECT_MSG = "Cache group restore operation was rejected. "; diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java index 9ab5170e8a1729..62f4619edb8b6a 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreBaseTest.java @@ -50,7 +50,7 @@ protected IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt) throws Exce * @throws Exception if failed. */ protected IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt, boolean startClient) throws Exception { - IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder(), dfltCacheCfg.setBackups(0)); + IgniteEx ignite = startGridsWithCache(nodesCnt, keysCnt, valueBuilder(), dfltCacheCfg); if (startClient) ignite = startClientGrid("client"); @@ -68,7 +68,7 @@ protected IgniteEx startGridsWithSnapshot(int nodesCnt, int keysCnt, boolean sta * @param cache Cache. * @param keysCnt Expected number of keys. */ - protected void checkCacheKeys(IgniteCache cache, int keysCnt) { + protected void assertCacheKeys(IgniteCache cache, int keysCnt) { assertEquals(keysCnt, cache.size()); for (int i = 0; i < keysCnt; i++) 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 16ab9f9f4d88d3..f3359f1e4d36e5 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 @@ -122,9 +122,9 @@ public void testRestoreAllGroups() throws Exception { // Restore all cache groups. grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, null).get(TIMEOUT); - checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); - checkCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); - checkCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -144,33 +144,32 @@ public void testStartClusterSnapshotRestoreMultipleThreadsDiffNode() throws Exce /** * @param nodeIdxSupplier Ignite node index supplier. */ - public void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSupplier) throws Exception { + private void checkStartClusterSnapshotRestoreMultithreaded(IntSupplier nodeIdxSupplier) throws Exception { Ignite ignite = startGridsWithSnapshot(2, CACHE_KEYS_RANGE); - CountDownLatch startLatch = new CountDownLatch(1); AtomicInteger successCnt = new AtomicInteger(); + AtomicInteger failCnt = new AtomicInteger(); IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(() -> { try { - startLatch.await(TIMEOUT, TimeUnit.MILLISECONDS); + nodeIdxSupplier.getAsInt(); grid(nodeIdxSupplier.getAsInt()).snapshot().restoreSnapshot( SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); successCnt.incrementAndGet(); } - catch (Exception ignore) { - // Expected exception. + catch (Exception e) { + failCnt.incrementAndGet(); } }, 2, "runner"); - startLatch.countDown(); - fut.get(TIMEOUT); assertEquals(1, successCnt.get()); + assertEquals(1, failCnt.get()); - checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -198,7 +197,7 @@ public void testCreateSnapshotDuringRestore() throws Exception { fut.get(TIMEOUT); - checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); } /** @@ -356,8 +355,8 @@ public void testRestoreSharedCacheGroup() throws Exception { snp.restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(SHARED_GRP)).get(TIMEOUT); - checkCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); - checkCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(CACHE1), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(CACHE2), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -383,10 +382,7 @@ public void testIncompatibleMetasUpdate() throws Exception { return objs[n]; }); - IgniteFuture fut = - ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)); - - fut.get(TIMEOUT); + ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); // Ensure that existing type has been updated. BinaryType type = ignite.context().cacheObjects().metadata(typeId); @@ -486,7 +482,7 @@ private void checkCacheStartWithTheSameName( fut.get(TIMEOUT); - checkCacheKeys(grid(0).cache(CACHE1), CACHE_KEYS_RANGE); + assertCacheKeys(grid(0).cache(CACHE1), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -602,7 +598,7 @@ public void testNodeJoinDuringRestore() throws Exception { assertTrue(cache.indexReadyFuture().isDone()); - checkCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(cache, CACHE_KEYS_RANGE); } /** @@ -669,7 +665,7 @@ private void checkClusterStateChange( ignite.cluster().state(ClusterState.ACTIVE); - checkCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); return; } @@ -684,7 +680,7 @@ private void checkClusterStateChange( grid(nodesCnt - 1).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(cacheName)).get(TIMEOUT); - checkCacheKeys(ignite.cache(cacheName), CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(cacheName), CACHE_KEYS_RANGE); } /** diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java index ff58afc4121970..8328939ef29233 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java @@ -76,7 +76,7 @@ public void testBasicClusterSnapshotRestore() throws Exception { assertTrue(cache.indexReadyFuture().isDone()); - checkCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(cache, CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -97,7 +97,10 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { assertTrue(cache.indexReadyFuture().isDone()); - checkCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(cache, CACHE_KEYS_RANGE); + + for (Ignite grid : G.allGrids()) + assertNotNull(((IgniteEx)grid).context().cacheObjects().metadata(typeId)); } /** @throws Exception If failed. */ @@ -138,12 +141,12 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { awaitPartitionMapExchange(); - checkCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(cache, CACHE_KEYS_RANGE); } /** {@inheritDoc} */ - @Override protected void checkCacheKeys(IgniteCache cache, int keysCnt) { - super.checkCacheKeys(cache, keysCnt); + @Override protected void assertCacheKeys(IgniteCache cache, int keysCnt) { + super.assertCacheKeys(cache, keysCnt); String tblName = new BinaryBasicNameMapper(true).typeName(TYPE_NAME); From dcb35abd1621ba59f8142c0c3eb5674dd0b1f188 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 29 Apr 2021 18:18:22 +0300 Subject: [PATCH 94/98] IGNITE-13805 Added DynamicCacheStartFailsOnNodeLeftTest. --- .../DynamicCacheStartFailsOnNodeLeftTest.java | 128 ++++++++++++++++++ 1 file changed, 128 insertions(+) create mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java new file mode 100644 index 00000000000000..ce9dfc3f71ed7b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java @@ -0,0 +1,128 @@ +/* + * 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; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.UUID; +import org.apache.ignite.Ignite; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.TestRecordingCommunicationSpi; +import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.internal.util.typedef.internal.CU; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +/** + * Test the dynamic cache startup failure feature if the required node leaves the topology during cache startup. + */ +public class DynamicCacheStartFailsOnNodeLeftTest extends GridCommonAbstractTest { + /** Test cache name, */ + private static final String TEST_CACHE = "testCache"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + return super.getConfiguration(igniteInstanceName).setCommunicationSpi(new TestRecordingCommunicationSpi()); + } + + /** {@inheritDoc} */ + @Override public void afterTest() throws Exception { + stopAllGrids(); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testRollbackOnRequiredTopologyWhenCrdFailsOnExhangeFinish() throws Exception { + checkRollbackOnRequiredTopologyWhenNodeFails(true); + } + + /** + * @throws Exception If failed. + */ + @Test + public void testRollbackOnRequiredTopologyWhenNodeFailsOnExhangeFinish() throws Exception { + checkRollbackOnRequiredTopologyWhenNodeFails(false); + } + + + /** + * @param crdStop {{@code True} to stop coordinator node. + */ + public void checkRollbackOnRequiredTopologyWhenNodeFails(boolean crdStop) throws Exception { + startGridsMultiThreaded(3); + + Collection srvNodes = F.viewReadOnly(grid(0).cluster().nodes(), F.node2id()); + + IgniteEx client = startClientGrid(); + + TestRecordingCommunicationSpi node1spi = TestRecordingCommunicationSpi.spi(grid(1)); + TestRecordingCommunicationSpi node2spi = TestRecordingCommunicationSpi.spi(grid(2)); + + node1spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); + node2spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); + + StoredCacheData storedConf = new StoredCacheData(new CacheConfiguration<>(TEST_CACHE)); + + IgniteInternalFuture fut = client.context().cache().dynamicStartCachesByStoredConf( + Collections.singleton(storedConf), true, true, false, null, true, new HashSet<>(srvNodes)); + + node1spi.waitForBlocked(); + node2spi.waitForBlocked(); + + stopGrid(crdStop ? 0 : 2, true); + + node1spi.stopBlock(); + + if (crdStop) + node2spi.stopBlock(); + + boolean exFound = false; + + try { + fut.get(getTestTimeout()); + } catch (Exception e) { + for (Throwable t : e.getSuppressed()) { + if (t.getClass().equals(ClusterTopologyCheckedException.class) && + t.getMessage().contains("Required node has left the cluster")) { + exFound = true; + + break; + } + } + } + + assertTrue(exFound); + + awaitPartitionMapExchange(); + + for (Ignite grid : G.allGrids()) + assertNull(((IgniteEx)grid).context().cache().cacheGroup(CU.cacheId(TEST_CACHE))); + + // Make sure the cache can be successfully created. + client.createCache(storedConf.config()); + } +} From 5747addb1c946bc30228b0037d7855c92184293f Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 29 Apr 2021 18:37:57 +0300 Subject: [PATCH 95/98] IGNITE-13805 Make sure no index rebuild happened. --- ...lusterSnapshotRestoreWithIndexingTest.java | 33 +++++++++---------- 1 file changed, 16 insertions(+), 17 deletions(-) diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java index 8328939ef29233..17f495c02046b3 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java +++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotRestoreWithIndexingTest.java @@ -29,6 +29,7 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.GridKernalContext; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.processors.query.GridQueryProcessor; import org.apache.ignite.internal.util.typedef.F; @@ -72,11 +73,7 @@ public void testBasicClusterSnapshotRestore() throws Exception { grid(0).snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = client.cache(DEFAULT_CACHE_NAME); - - assertTrue(cache.indexReadyFuture().isDone()); - - assertCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(client.cache(DEFAULT_CACHE_NAME), CACHE_KEYS_RANGE); } /** @throws Exception If failed. */ @@ -93,11 +90,7 @@ public void testBasicClusterSnapshotRestoreWithMetadata() throws Exception { ignite.snapshot().restoreSnapshot(SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(); - - assertTrue(cache.indexReadyFuture().isDone()); - - assertCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(), CACHE_KEYS_RANGE); for (Ignite grid : G.allGrids()) assertNotNull(((IgniteEx)grid).context().cacheObjects().metadata(typeId)); @@ -135,13 +128,9 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { ignite.snapshot().restoreSnapshot( SNAPSHOT_NAME, Collections.singleton(DEFAULT_CACHE_NAME)).get(TIMEOUT); - IgniteCache cache = ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(); - - assertTrue(cache.indexReadyFuture().isDone()); - awaitPartitionMapExchange(); - assertCacheKeys(cache, CACHE_KEYS_RANGE); + assertCacheKeys(ignite.cache(DEFAULT_CACHE_NAME).withKeepBinary(), CACHE_KEYS_RANGE); } /** {@inheritDoc} */ @@ -151,17 +140,27 @@ public void testClusterSnapshotRestoreOnBiggerTopology() throws Exception { String tblName = new BinaryBasicNameMapper(true).typeName(TYPE_NAME); for (Ignite grid : G.allGrids()) { + GridKernalContext ctx = ((IgniteEx)grid).context(); + + String nodeId = ctx.localNodeId().toString(); + + assertTrue("nodeId=" + nodeId, grid.cache(cache.getName()).indexReadyFuture().isDone()); + + // Make sure no index rebuild happened. + assertEquals("nodeId=" + nodeId, + 0, ctx.cache().cache(cache.getName()).context().cache().metrics0().getIndexRebuildKeysProcessed()); + GridQueryProcessor qry = ((IgniteEx)grid).context().query(); // Make sure SQL works fine. - assertEquals((long)keysCnt, qry.querySqlFields(new SqlFieldsQuery( + assertEquals("nodeId=" + nodeId, (long)keysCnt, qry.querySqlFields(new SqlFieldsQuery( "SELECT count(*) FROM " + tblName), true).getAll().get(0).get(0)); // Make sure the index is in use. String explainPlan = (String)qry.querySqlFields(new SqlFieldsQuery( "explain SELECT * FROM " + tblName + " WHERE id < 10"), true).getAll().get(0).get(0); - assertTrue("id=" + grid.cluster().localNode().id() + "\n" + explainPlan, explainPlan.contains("ID_ASC_IDX")); + assertTrue("nodeId=" + nodeId + "\n" + explainPlan, explainPlan.contains("ID_ASC_IDX")); } } From 8f138d00fb526117fedfc2e38f92117c8c8a2420 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 29 Apr 2021 21:33:03 +0300 Subject: [PATCH 96/98] IGNITE-13805 (minor) Javadoc improved, cache start test included into suite. --- .../core/src/main/java/org/apache/ignite/IgniteSnapshot.java | 3 ++- .../cache/DynamicCacheStartFailsOnNodeLeftTest.java | 5 ++--- .../org/apache/ignite/testsuites/IgniteCacheTestSuite8.java | 3 +++ 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java index 466ed24a7d12d2..0d46cbdc488210 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java @@ -54,7 +54,8 @@ public interface IgniteSnapshot { /** * Restore cache group(s) from the snapshot. *

- * NOTE: Cache groups to be restored from the snapshot must not present in the cluster, if they present, they must be destroyed by the user before starting this operation. + * NOTE: Cache groups to be restored from the snapshot must not present in the cluster, if they present, + * they must be destroyed by the user (eg with {@link IgniteCache#destroy()}) before starting this operation. * * @param name Snapshot name. * @param cacheGroupNames Cache groups to be restored or {@code null} to restore all cache groups from the snapshot. diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java index ce9dfc3f71ed7b..3d0e6021bc0744 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java @@ -56,7 +56,7 @@ public class DynamicCacheStartFailsOnNodeLeftTest extends GridCommonAbstractTest * @throws Exception If failed. */ @Test - public void testRollbackOnRequiredTopologyWhenCrdFailsOnExhangeFinish() throws Exception { + public void testRollbackOnRequiredTopologyWhenCrdFailsOnExhange() throws Exception { checkRollbackOnRequiredTopologyWhenNodeFails(true); } @@ -64,11 +64,10 @@ public void testRollbackOnRequiredTopologyWhenCrdFailsOnExhangeFinish() throws E * @throws Exception If failed. */ @Test - public void testRollbackOnRequiredTopologyWhenNodeFailsOnExhangeFinish() throws Exception { + public void testRollbackOnRequiredTopologyWhenNodeFailsOnExhange() throws Exception { checkRollbackOnRequiredTopologyWhenNodeFails(false); } - /** * @param crdStop {{@code True} to stop coordinator node. */ diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java index 43784b39d8251d..308f3b9dfe418f 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java @@ -22,6 +22,7 @@ import java.util.List; import org.apache.ignite.cache.ClientCreateCacheGroupOnJoinNodeMapsTest; import org.apache.ignite.internal.processors.cache.CacheStoreTxPutAllMultiNodeTest; +import org.apache.ignite.internal.processors.cache.DynamicCacheStartFailsOnNodeLeftTest; import org.apache.ignite.internal.processors.cache.GridCacheOrderedPreloadingSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRabalancingDelayedPartitionMapExchangeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalanceOrderTest; @@ -80,6 +81,8 @@ public static List> suite(Collection ignoredTests) { GridTestUtils.addTestIfNeeded(suite, CleanupRestoredCachesSlowTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, ClientCreateCacheGroupOnJoinNodeMapsTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, DynamicCacheStartFailsOnNodeLeftTest.class, ignoredTests); + return suite; } } From 59dcefa782f0696aa87996b501dd790e6326bd75 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Thu, 13 May 2021 21:24:14 +0300 Subject: [PATCH 97/98] IGNITE-13805 Use restartId instead of internal flag. --- .../apache/ignite/internal/IgniteKernal.java | 2 - .../processors/cache/ClusterCachesInfo.java | 61 ++++----- .../cache/DynamicCacheChangeBatch.java | 20 +-- .../cache/DynamicCacheChangeRequest.java | 17 --- .../processors/cache/GridCacheProcessor.java | 68 ++-------- .../GridDhtPartitionsExchangeFuture.java | 2 +- .../snapshot/IgniteSnapshotManager.java | 17 +++ .../snapshot/SnapshotRestoreProcess.java | 17 ++- .../DynamicCacheStartFailsOnNodeLeftTest.java | 127 ------------------ .../IgniteDynamicCacheStartSelfTest.java | 4 +- .../testsuites/IgniteCacheTestSuite8.java | 3 - 11 files changed, 77 insertions(+), 261 deletions(-) delete mode 100644 modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 7b43a5f2fd7c50..f280ce541e24f8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -3240,7 +3240,6 @@ public IgniteInternalCache getCache(String name) { ctx.cache().dynamicStartCaches(cacheCfgs, true, true, - false, false).get(); List createdCaches = new ArrayList<>(cacheCfgs.size()); @@ -3337,7 +3336,6 @@ public IgniteInternalCache getCache(String name) { ctx.cache().dynamicStartCaches(cacheCfgs, false, true, - false, false).get(); List createdCaches = new ArrayList<>(cacheCfgs.size()); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 14851d8a46fa00..83e880cfddc166 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -583,24 +583,10 @@ public void onCacheChangeRequested(DynamicCacheChangeFailureMessage failMsg, Aff public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) { DiscoveryDataClusterState state = ctx.state().clusterState(); - if (!state.active() || state.transition()) { - IgniteCheckedException err = new IgniteCheckedException("Failed to start/stop cache, cluster state change " + - "is in progress."); - - for (DynamicCacheChangeRequest req : batch.requests()) { - if (req.template()) { - ctx.cache().completeTemplateAddFuture(req.startCacheConfiguration().getName(), - req.deploymentId()); - } - else - ctx.cache().completeCacheStartFuture(req, false, err); - } - - return false; - } + if (state.active() && !state.transition()) { + Collection nodes = ctx.cache().context().snapshotMgr().cacheStartRequiredAliveNodes(batch.requests()); - if (!F.isEmpty(batch.topologyNodes())) { - for (UUID nodeId : batch.topologyNodes()) { + for (UUID nodeId : nodes) { ClusterNode node = ctx.discovery().node(nodeId); if (node != null && CU.baselineNode(node, state) && ctx.discovery().alive(node)) @@ -614,25 +600,40 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop return false; } - } - ExchangeActions exchangeActions = new ExchangeActions(); + ExchangeActions exchangeActions = new ExchangeActions(); - CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions, - batch.requests(), - topVer, - false); + CacheChangeProcessResult res = processCacheChangeRequests(exchangeActions, + batch.requests(), + topVer, + false); + + if (res.needExchange) { + assert !exchangeActions.empty() : exchangeActions; - if (res.needExchange) { - assert !exchangeActions.empty() : exchangeActions; + batch.exchangeActions(exchangeActions); - batch.exchangeActions(exchangeActions); + if (!F.isEmpty(nodes)) + exchangeActions.cacheStartRequiredAliveNodes(nodes); + } - if (!F.isEmpty(batch.topologyNodes())) - exchangeActions.cacheStartRequiredAliveNodes(batch.topologyNodes()); + return res.needExchange; } + else { + IgniteCheckedException err = new IgniteCheckedException("Failed to start/stop cache, cluster state change " + + "is in progress."); - return res.needExchange; + for (DynamicCacheChangeRequest req : batch.requests()) { + if (req.template()) { + ctx.cache().completeTemplateAddFuture(req.startCacheConfiguration().getName(), + req.deploymentId()); + } + else + ctx.cache().completeCacheStartFuture(req, false, err); + } + + return false; + } } /** @@ -1028,7 +1029,7 @@ else if (encMgr.masterKeyDigest() != null && } } - if (err == null && !req.internal()) { + if (err == null && req.restartId() == null) { IgniteSnapshotManager snapshotMgr = ctx.cache().context().snapshotMgr(); if (snapshotMgr.isRestoring(cacheName, ccfg.getGroupName())) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index a38c006a20eae8..6b89424462b0d6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -19,7 +19,6 @@ import java.util.Collection; import java.util.Set; -import java.util.UUID; import org.apache.ignite.internal.managers.discovery.DiscoCache; import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager; @@ -46,12 +45,6 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { @GridToStringInclude private Collection reqs; - /** - * Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when starting - * the cache(s), the whole procedure is rolled back. - */ - private Collection rqNodes; - /** Cache updates to be executed on exchange. */ private transient ExchangeActions exchangeActions; @@ -67,14 +60,11 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { /** * @param reqs Requests. - * @param rqNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails - * when starting the cache(s), the whole procedure is rolled back. */ - public DynamicCacheChangeBatch(Collection reqs, @Nullable Collection rqNodes) { + public DynamicCacheChangeBatch(Collection reqs) { assert !F.isEmpty(reqs) : reqs; this.reqs = reqs; - this.rqNodes = rqNodes; } /** {@inheritDoc} */ @@ -172,14 +162,6 @@ public void startCaches(boolean startCaches) { this.startCaches = startCaches; } - /** - * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when - * starting the cache(s), the whole procedure is rolled back. - */ - public Collection topologyNodes() { - return rqNodes; - } - /** {@inheritDoc} */ @Override public String toString() { return S.toString(DynamicCacheChangeBatch.class, this); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java index a97ca08d3b97f7..88d44cc23090f9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java @@ -107,9 +107,6 @@ public class DynamicCacheChangeRequest implements Serializable { /** Cache configuration enrichment. */ private CacheConfigurationEnrichment cacheCfgEnrichment; - /** Flag indicating that the cache was started internally and not by the user. */ - private boolean internal; - /** * @param reqId Unique request ID. * @param cacheName Cache stop name. @@ -483,20 +480,6 @@ public void masterKeyDigest(@Nullable byte[] masterKeyDigest) { return masterKeyDigest; } - /** - * @param internal Flag indicating that the cache was started internally and not by the user. - */ - public void internal(boolean internal) { - this.internal = internal; - } - - /** - * @return Flag indicating that the cache was started internally and not by the user. - */ - public boolean internal() { - return internal; - } - /** * @return Cache configuration enrichment. */ diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index 9bd48d17dac6f0..00b56b83cb632b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -3520,8 +3520,7 @@ public IgniteInternalFuture dynamicStartCache( false, null, ccfg != null && ccfg.isEncryptionEnabled() ? grpKeys.iterator().next() : null, - ccfg != null && ccfg.isEncryptionEnabled() ? masterKeyDigest : null, - false); + ccfg != null && ccfg.isEncryptionEnabled() ? masterKeyDigest : null); if (req != null) { if (req.clientStartOnly()) @@ -3709,46 +3708,20 @@ private IgniteInternalFuture startClientCacheChange( * @param failIfExists Fail if exists flag. * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. - * @param internal Flag indicating that the cache was started internally and not by the user. * @return Future that will be completed when all caches are deployed. */ public IgniteInternalFuture dynamicStartCaches( Collection ccfgList, boolean failIfExists, boolean checkThreadTx, - boolean disabledAfterStart, - boolean internal + boolean disabledAfterStart ) { return dynamicStartCachesByStoredConf( ccfgList.stream().map(StoredCacheData::new).collect(toList()), failIfExists, checkThreadTx, disabledAfterStart, - null, - internal); - } - - /** - * Dynamically starts multiple caches. - * - * @param storedCacheDataList Collection of stored cache data. - * @param failIfExists Fail if exists flag. - * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. - * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. - * @param restartId Restart requester id (it'll allow to start this cache only him). - * @param internal Flag indicating that the cache was started internally and not by the user. - * @return Future that will be completed when all caches are deployed. - */ - public IgniteInternalFuture dynamicStartCachesByStoredConf( - Collection storedCacheDataList, - boolean failIfExists, - boolean checkThreadTx, - boolean disabledAfterStart, - IgniteUuid restartId, - boolean internal - ) { - return dynamicStartCachesByStoredConf(storedCacheDataList, failIfExists, checkThreadTx, disabledAfterStart, - restartId, internal, null); + null); } /** @@ -3759,9 +3732,6 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( * @param checkThreadTx If {@code true} checks that current thread does not have active transactions. * @param disabledAfterStart If true, cache proxies will be only activated after {@link #restartProxies()}. * @param restartId Restart requester id (it'll allow to start this cache only him). - * @param internal Flag indicating that the cache was started internally and not by the user. - * @param topNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails - * when starting the cache(s), the whole procedure is rolled back. * @return Future that will be completed when all caches are deployed. */ public IgniteInternalFuture dynamicStartCachesByStoredConf( @@ -3769,9 +3739,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( boolean failIfExists, boolean checkThreadTx, boolean disabledAfterStart, - IgniteUuid restartId, - boolean internal, - @Nullable Set topNodes + IgniteUuid restartId ) { if (checkThreadTx) { sharedCtx.tm().checkEmptyTransactions(() -> { @@ -3806,8 +3774,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( disabledAfterStart, ccfg.queryEntities(), ccfg.config().isEncryptionEnabled() ? grpKeysIter.next() : null, - ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null, - internal); + ccfg.config().isEncryptionEnabled() ? masterKeyDigest : null); if (req != null) { if (req.clientStartOnly()) { @@ -3833,7 +3800,7 @@ public IgniteInternalFuture dynamicStartCachesByStoredConf( GridCompoundFuture compoundFut = new GridCompoundFuture<>(); - for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs, topNodes)) + for (DynamicCacheStartFuture fut : initiateCacheChanges(srvReqs)) compoundFut.add((IgniteInternalFuture)fut); if (clientReqs != null) { @@ -4086,19 +4053,6 @@ public void cleanupCachesDirectories() throws IgniteCheckedException { */ private Collection initiateCacheChanges( Collection reqs - ) { - return initiateCacheChanges(reqs, null); - } - - /** - * @param reqs Requests. - * @param rqNodes Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails - * when starting the cache(s), the whole procedure is rolled back. - * @return Collection of futures. - */ - private Collection initiateCacheChanges( - Collection reqs, - @Nullable Collection rqNodes ) { Collection res = new ArrayList<>(reqs.size()); @@ -4154,7 +4108,7 @@ private Collection initiateCacheChanges( if (!sndReqs.isEmpty()) { try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs, rqNodes)); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs)); err = checkNodeState(); } @@ -4719,7 +4673,7 @@ public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteChec Exception err = null; try { - ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req), null)); + ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req))); if (ctx.isStopping()) { err = new IgniteCheckedException("Failed to execute dynamic cache change request, " + @@ -5091,7 +5045,6 @@ private T withBinaryContext(IgniteOutClosureX c) throws IgniteCheckedExce * @param qryEntities Query entities. * @param encKey Encryption key. * @param masterKeyDigest Master key digest. - * @param internal Flag indicating that the cache was started internally and not by the user. * @return Request or {@code null} if cache already exists. * @throws IgniteCheckedException if some of pre-checks failed * @throws CacheExistsException if cache exists and failIfExists flag is {@code true} @@ -5108,8 +5061,7 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( boolean disabledAfterStart, @Nullable Collection qryEntities, @Nullable byte[] encKey, - @Nullable byte[] masterKeyDigest, - boolean internal + @Nullable byte[] masterKeyDigest ) throws IgniteCheckedException { DynamicCacheDescriptor desc = cacheDescriptor(cacheName); @@ -5127,8 +5079,6 @@ private DynamicCacheChangeRequest prepareCacheChangeRequest( req.restartId(restartId); - req.internal(internal); - if (ccfg != null) { cloneCheckSerializable(ccfg); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java index 7e3f465d473f97..6ceab2f83f35ef 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java @@ -1162,7 +1162,7 @@ private void ensureClientCachesStarted() { } if (!notStartedCacheConfigs.isEmpty()) - cacheProcessor.dynamicStartCaches(notStartedCacheConfigs, false, false, false, false); + cacheProcessor.dynamicStartCaches(notStartedCacheConfigs, false, false, false); } /** 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 5ff56aadbd36ce..c4cce9a39cf1dd 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 @@ -79,6 +79,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.CacheType; +import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; @@ -128,6 +129,7 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteCallable; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.marshaller.Marshaller; import org.apache.ignite.marshaller.MarshallerUtils; import org.apache.ignite.resources.IgniteInstanceResource; @@ -776,6 +778,21 @@ public boolean isRestoring() { return restoreCacheGrpProc.isRestoring(); } + /** + * @param reqs Collection of cache change requests. + * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when + * starting the cache(s), the whole procedure is rolled back. + */ + public Set cacheStartRequiredAliveNodes(Collection reqs) { + IgniteUuid restoreId = F.first( + F.viewReadOnly(reqs, DynamicCacheChangeRequest::restartId, req -> req.start() && req.restartId() != null)); + + if (restoreId == null) + return Collections.emptySet(); + + return restoreCacheGrpProc.cacheStartRequiredAliveNodes(restoreId); + } + /** * Check if the cache or group with the specified name is currently being restored from the snapshot. * 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 1398109beb7999..038561bef39ade 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 @@ -64,6 +64,7 @@ import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteFuture; +import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; import static org.apache.ignite.internal.IgniteFeatures.SNAPSHOT_RESTORE_CACHE_GROUP; @@ -319,6 +320,20 @@ else if (CU.cacheId(locGrpName) == cacheId) return false; } + /** + * @param reqId Request ID. + * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when + * starting the cache(s), the whole procedure is rolled back. + */ + public Set cacheStartRequiredAliveNodes(IgniteUuid reqId) { + SnapshotRestoreContext opCtx0 = opCtx; + + if (opCtx0 == null || !reqId.globalId().equals(opCtx0.reqId)) + return Collections.emptySet(); + + return Collections.unmodifiableSet(opCtx0.nodes); + } + /** * Finish local cache group restore process. * @@ -736,7 +751,7 @@ private IgniteInternalFuture cacheStart(UUID reqId) { // We set the topology node IDs required to successfully start the cache, if any of the required nodes leave // the cluster during the cache startup, the whole procedure will be rolled back. - return ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, null, true, opCtx0.nodes); + return ctx.cache().dynamicStartCachesByStoredConf(ccfgs, true, true, false, IgniteUuid.fromUuid(reqId)); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java deleted file mode 100644 index 3d0e6021bc0744..00000000000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DynamicCacheStartFailsOnNodeLeftTest.java +++ /dev/null @@ -1,127 +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; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.UUID; -import org.apache.ignite.Ignite; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.TestRecordingCommunicationSpi; -import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage; -import org.apache.ignite.internal.util.typedef.F; -import org.apache.ignite.internal.util.typedef.G; -import org.apache.ignite.internal.util.typedef.internal.CU; -import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; -import org.junit.Test; - -/** - * Test the dynamic cache startup failure feature if the required node leaves the topology during cache startup. - */ -public class DynamicCacheStartFailsOnNodeLeftTest extends GridCommonAbstractTest { - /** Test cache name, */ - private static final String TEST_CACHE = "testCache"; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { - return super.getConfiguration(igniteInstanceName).setCommunicationSpi(new TestRecordingCommunicationSpi()); - } - - /** {@inheritDoc} */ - @Override public void afterTest() throws Exception { - stopAllGrids(); - } - - /** - * @throws Exception If failed. - */ - @Test - public void testRollbackOnRequiredTopologyWhenCrdFailsOnExhange() throws Exception { - checkRollbackOnRequiredTopologyWhenNodeFails(true); - } - - /** - * @throws Exception If failed. - */ - @Test - public void testRollbackOnRequiredTopologyWhenNodeFailsOnExhange() throws Exception { - checkRollbackOnRequiredTopologyWhenNodeFails(false); - } - - /** - * @param crdStop {{@code True} to stop coordinator node. - */ - public void checkRollbackOnRequiredTopologyWhenNodeFails(boolean crdStop) throws Exception { - startGridsMultiThreaded(3); - - Collection srvNodes = F.viewReadOnly(grid(0).cluster().nodes(), F.node2id()); - - IgniteEx client = startClientGrid(); - - TestRecordingCommunicationSpi node1spi = TestRecordingCommunicationSpi.spi(grid(1)); - TestRecordingCommunicationSpi node2spi = TestRecordingCommunicationSpi.spi(grid(2)); - - node1spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); - node2spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage); - - StoredCacheData storedConf = new StoredCacheData(new CacheConfiguration<>(TEST_CACHE)); - - IgniteInternalFuture fut = client.context().cache().dynamicStartCachesByStoredConf( - Collections.singleton(storedConf), true, true, false, null, true, new HashSet<>(srvNodes)); - - node1spi.waitForBlocked(); - node2spi.waitForBlocked(); - - stopGrid(crdStop ? 0 : 2, true); - - node1spi.stopBlock(); - - if (crdStop) - node2spi.stopBlock(); - - boolean exFound = false; - - try { - fut.get(getTestTimeout()); - } catch (Exception e) { - for (Throwable t : e.getSuppressed()) { - if (t.getClass().equals(ClusterTopologyCheckedException.class) && - t.getMessage().contains("Required node has left the cluster")) { - exFound = true; - - break; - } - } - } - - assertTrue(exFound); - - awaitPartitionMapExchange(); - - for (Ignite grid : G.allGrids()) - assertNull(((IgniteEx)grid).context().cache().cacheGroup(CU.cacheId(TEST_CACHE))); - - // Make sure the cache can be successfully created. - client.createCache(storedConf.config()); - } -} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java index a1bc9e8359dd98..7648ec6650bc80 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java @@ -1440,7 +1440,7 @@ public void testCacheRestartIsAllowedOnlyToItsInititator() throws Exception { StoredCacheData storedCacheData = new StoredCacheData(ccfg); try { - kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, IgniteUuid.randomUuid(), false).get(); + kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, IgniteUuid.randomUuid()).get(); fail(); } @@ -1450,7 +1450,7 @@ public void testCacheRestartIsAllowedOnlyToItsInititator() throws Exception { System.out.println("We couldn't start new cache with wrong restart id."); } - kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, restartId, false).get(); + kernal.context().cache().dynamicStartCachesByStoredConf(Collections.singleton(storedCacheData), true, true, false, restartId).get(); System.out.println("We successfully restarted cache with initial restartId."); diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java index 308f3b9dfe418f..43784b39d8251d 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite8.java @@ -22,7 +22,6 @@ import java.util.List; import org.apache.ignite.cache.ClientCreateCacheGroupOnJoinNodeMapsTest; import org.apache.ignite.internal.processors.cache.CacheStoreTxPutAllMultiNodeTest; -import org.apache.ignite.internal.processors.cache.DynamicCacheStartFailsOnNodeLeftTest; import org.apache.ignite.internal.processors.cache.GridCacheOrderedPreloadingSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRabalancingDelayedPartitionMapExchangeSelfTest; import org.apache.ignite.internal.processors.cache.distributed.rebalancing.GridCacheRebalanceOrderTest; @@ -81,8 +80,6 @@ public static List> suite(Collection ignoredTests) { GridTestUtils.addTestIfNeeded(suite, CleanupRestoredCachesSlowTest.class, ignoredTests); GridTestUtils.addTestIfNeeded(suite, ClientCreateCacheGroupOnJoinNodeMapsTest.class, ignoredTests); - GridTestUtils.addTestIfNeeded(suite, DynamicCacheStartFailsOnNodeLeftTest.class, ignoredTests); - return suite; } } From 4ee0cf63ba19518ab43a7e51ef1ee38331d36a73 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 14 May 2021 14:07:36 +0300 Subject: [PATCH 98/98] IGNITE-13805 Code cleanup. --- .../internal/processors/cache/ClusterCachesInfo.java | 9 +++++++-- .../persistence/snapshot/IgniteSnapshotManager.java | 8 ++------ 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index 83e880cfddc166..236051a974ee53 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -584,7 +584,12 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop DiscoveryDataClusterState state = ctx.state().clusterState(); if (state.active() && !state.transition()) { - Collection nodes = ctx.cache().context().snapshotMgr().cacheStartRequiredAliveNodes(batch.requests()); + Set restartIds = new HashSet<>(F.viewReadOnly( + batch.requests(), DynamicCacheChangeRequest::restartId, req -> req.start() && req.restartId() != null)); + + assert restartIds.size() <= 1 : batch.requests(); + + Collection nodes = ctx.cache().context().snapshotMgr().cacheStartRequiredAliveNodes(F.first(restartIds)); for (UUID nodeId : nodes) { ClusterNode node = ctx.discovery().node(nodeId); @@ -613,7 +618,7 @@ public boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTop batch.exchangeActions(exchangeActions); - if (!F.isEmpty(nodes)) + if (!nodes.isEmpty()) exchangeActions.cacheStartRequiredAliveNodes(nodes); } 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 c4cce9a39cf1dd..90304bb0758bed 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 @@ -79,7 +79,6 @@ import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor; import org.apache.ignite.internal.processors.cache.CacheObjectContext; import org.apache.ignite.internal.processors.cache.CacheType; -import org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture; @@ -779,14 +778,11 @@ public boolean isRestoring() { } /** - * @param reqs Collection of cache change requests. + * @param restoreId Restore process ID. * @return Server nodes on which a successful start of the cache(s) is required, if any of these nodes fails when * starting the cache(s), the whole procedure is rolled back. */ - public Set cacheStartRequiredAliveNodes(Collection reqs) { - IgniteUuid restoreId = F.first( - F.viewReadOnly(reqs, DynamicCacheChangeRequest::restartId, req -> req.start() && req.restartId() != null)); - + public Set cacheStartRequiredAliveNodes(@Nullable IgniteUuid restoreId) { if (restoreId == null) return Collections.emptySet();