alex-plekhanov commented on a change in pull request #6554: IGNITE-11073: 
Backup page store manager, initial
URL: https://github.com/apache/ignite/pull/6554#discussion_r393167458
 
 

 ##########
 File path: 
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
 ##########
 @@ -0,0 +1,884 @@
+/*
+ * 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.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+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;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicIntegerArray;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BooleanSupplier;
+import java.util.function.Consumer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.pagemem.store.PageWriteListener;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import 
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
+import 
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
+import 
org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology;
+import 
org.apache.ignite.internal.processors.cache.persistence.DbCheckpointListener;
+import 
org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+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.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
+import org.apache.ignite.internal.processors.marshaller.MappedName;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.lang.IgniteThrowableRunner;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+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;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import static 
org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+import static 
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirName;
+import static 
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheWorkDir;
+import static 
org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.getPartitionFile;
+import static 
org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.partDeltaFile;
+import static 
org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.relativeNodePath;
+
+/**
+ *
+ */
+class SnapshotFutureTask extends GridFutureAdapter<Boolean> implements 
DbCheckpointListener {
+    /** Shared context. */
+    private final GridCacheSharedContext<?, ?> cctx;
+
+    /** Ignite logger */
+    private final IgniteLogger log;
+
+    /** Node id which cause snapshot operation. */
+    private final UUID srcNodeId;
+
+    /** Unique identifier of snapshot process. */
+    private final String snpName;
+
+    /** Snapshot working directory on file system. */
+    private final File tmpTaskWorkDir;
+
+    /** Local buffer to perpform copy-on-write operations for {@link 
PageStoreSerialWriter}. */
+    private final ThreadLocal<ByteBuffer> localBuff;
+
+    /** IO factory which will be used for creating snapshot delta-writers. */
+    private final FileIOFactory ioFactory;
+
+    /**
+     * The length of file size per each cache partiton file.
+     * Partition has value greater than zero only for partitons in OWNING 
state.
+     * Information collected under checkpoint write lock.
+     */
+    private final Map<GroupPartitionId, Long> partFileLengths = new 
HashMap<>();
+
+    /**
+     * Map of partitions to snapshot and theirs corresponding delta PageStores.
+     * Writers are pinned to the snapshot context due to controlling partition
+     * processing supplier.
+     */
+    private final Map<GroupPartitionId, PageStoreSerialWriter> 
partDeltaWriters = new HashMap<>();
+
+    /** Snapshot data sender. */
+    @GridToStringExclude
+    private final SnapshotFileSender snpSndr;
+
+    /**
+     * Initial map of cache groups and its partitions to include into 
snapshot. If array of partitions
+     * is {@code null} than all OWNING partitions for given cache groups will 
be included into snapshot.
+     * In this case if all of partitions have OWNING state the index partition 
also will be included.
+     * <p>
+     * If partitions for particular cache group are not provided that they 
will be collected and added
+     * on checkpoint under the write lock.
+     */
+    private final Map<Integer, Set<Integer>> parts;
+
+    /** Cache group and corresponding partitions collected under the 
checkpoint write lock. */
+    private final Map<Integer, Set<Integer>> processed = new HashMap<>();
+
+    /** Checkpoint end future. */
+    private final CompletableFuture<Boolean> cpEndFut = new 
CompletableFuture<>();
+
+    /** Future to wait until checkpoint mark pahse will be finished and 
snapshot tasks scheduled. */
+    private final GridFutureAdapter<Void> startedFut = new 
GridFutureAdapter<>();
+
+    /** Absolute snapshot storage path. */
+    private File tmpSnpDir;
+
+    /** {@code true} if operation has been cancelled. */
+    private volatile boolean cancelled;
+
+    /** An exception which has been ocurred during snapshot processing. */
+    private final AtomicReference<Throwable> err = new AtomicReference<>();
+
+    /** Flag indicates that task already scheduled on checkpoint. */
+    private final AtomicBoolean started = new AtomicBoolean();
+
+    /** Flag indicates the task must be interrupted. */
+    private final BooleanSupplier stopping = () -> cancelled || err.get() != 
null;
+
+    /**
+     * @param e Finished snapshot tosk future with particular exception.
+     */
+    public SnapshotFutureTask(IgniteCheckedException e) {
+        A.notNull(e, "Exception for a finished snapshot task must be not 
null");
+
+        cctx = null;
+        log = null;
+        snpName = null;
+        srcNodeId = null;
+        tmpTaskWorkDir = null;
+        snpSndr = null;
+
+        err.set(e);
+        startedFut.onDone(e);
+        onDone(e);
+        parts = null;
+        ioFactory = null;
+        localBuff = null;
+    }
+
+    /**
+     * @param snpName Unique identifier of snapshot task.
+     * @param ioFactory Factory to working with delta as file storage.
+     * @param parts Map of cache groups and its partitions to include into 
snapshot, if array of partitions
+     * is {@code null} than all OWNING partitions for given cache groups will 
be included into snapshot.
+     */
+    public SnapshotFutureTask(
+        GridCacheSharedContext<?, ?> cctx,
+        UUID srcNodeId,
+        String snpName,
+        File tmpWorkDir,
+        FileIOFactory ioFactory,
+        SnapshotFileSender snpSndr,
+        Map<Integer, Set<Integer>> parts,
+        ThreadLocal<ByteBuffer> localBuff
+    ) {
+        A.notNull(snpName, "Snapshot name cannot be empty or null");
+        A.notNull(snpSndr, "Snapshot sender which handles execution tasks must 
be not null");
+        A.notNull(snpSndr.executor(), "Executor service must be not null");
+
+        this.parts = parts;
+        this.cctx = cctx;
+        this.log = cctx.logger(SnapshotFutureTask.class);
+        this.snpName = snpName;
+        this.srcNodeId = srcNodeId;
+        this.tmpTaskWorkDir = new File(tmpWorkDir, snpName);
+        this.snpSndr = snpSndr;
+        this.ioFactory = ioFactory;
+        this.localBuff = localBuff;
+    }
+
+    /**
+     * @return Node id which triggers this operation..
+     */
+    public UUID sourceNodeId() {
+        return srcNodeId;
+    }
+
+    /**
+     * @return Type of snapshot operation.
+     */
+    public Class<? extends SnapshotFileSender> type() {
+        return snpSndr.getClass();
+    }
+
+    /**
+     * @return List of partitions to be processed.
+     */
+    public Set<Integer> affectedCacheGroups() {
+        return parts.keySet();
+    }
+
+    /**
+     * @param th An exception which occurred during snapshot processing.
+     */
+    public void acceptException(Throwable th) {
+        if (th == null)
+            return;
+
+        if (err.compareAndSet(null, th))
+            closeAsync();
+
+        startedFut.onDone(th);
+
+        log.error("Exception occurred during snapshot operation", th);
+    }
+
+    /**
+     * Close snapshot operation and release resources being used.
+     */
+    private void close() {
+        if (isDone())
+            return;
+
+        Throwable err0 = err.get();
+
+        if (onDone(true, err0, cancelled)) {
+            for (PageStoreSerialWriter writer : partDeltaWriters.values())
+                U.closeQuiet(writer);
+
+            snpSndr.close(err0);
+
+            if (tmpSnpDir != null)
+                U.delete(tmpSnpDir);
+
+            // Delete snapshot directory if no other files exists.
+            try {
+                if (U.fileCount(tmpTaskWorkDir.toPath()) == 0 || err0 != null)
+                    U.delete(tmpTaskWorkDir.toPath());
+            }
+            catch (IOException e) {
+                log.error("Snapshot directory doesn't exist [snpName=" + 
snpName + ", dir=" + tmpTaskWorkDir + ']');
+            }
+
+            if (err0 != null)
+                startedFut.onDone(err0);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If fails.
+     */
+    public void awaitStarted() throws IgniteCheckedException {
+        startedFut.get();
+    }
+
+    /**
+     * Initiates snapshot task.
+     */
+    public void start() {
+        if (stopping.getAsBoolean())
+            return;
+
+        try {
+            if (!started.compareAndSet(false, true))
+                return;
+
+            tmpSnpDir = 
U.resolveWorkDirectory(tmpTaskWorkDir.getAbsolutePath(),
+                
relativeNodePath(cctx.kernalContext().pdsFolderResolver().resolveFolders()),
+                false);
+
+            snpSndr.init();
+
+            for (Integer grpId : parts.keySet()) {
+                CacheGroupContext gctx = cctx.cache().cacheGroup(grpId);
+
+                if (gctx == null)
+                    throw new IgniteCheckedException("Cache group context has 
not found. Cache group is stopped: " + grpId);
+
+                if (!CU.isPersistentCache(gctx.config(), 
cctx.kernalContext().config().getDataStorageConfiguration()))
+                    throw new IgniteCheckedException("In-memory cache groups 
are not allowed to be snapshotted: " + grpId);
+
+                if (gctx.config().isEncryptionEnabled())
+                    throw new IgniteCheckedException("Encrypted cache groups 
are not allowed to be snapshotted: " + grpId);
+
+                // Create cache group snapshot directory on start in a single 
thread.
+                U.ensureDirectory(cacheWorkDir(tmpSnpDir, 
cacheDirName(gctx.config())),
+                    "directory for snapshotting cache group",
+                    log);
+            }
+
+            startedFut.listen(f ->
+                
((GridCacheDatabaseSharedManager)cctx.database()).removeCheckpointListener(this)
+            );
+
+            // Listener will be removed right after first execution
+            
((GridCacheDatabaseSharedManager)cctx.database()).addCheckpointListener(this);
+
+            if (log.isInfoEnabled()) {
+                log.info("Snapshot operation is scheduled on local node and 
will be handled by the checkpoint " +
+                    "listener [sctx=" + this + ", topVer=" + 
cctx.discovery().topologyVersionEx() + ']');
+            }
+        }
+        catch (IgniteCheckedException e) {
+            acceptException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void beforeCheckpointBegin(Context ctx) {
+        if (stopping.getAsBoolean())
+            return;
+
+        ctx.finishedStateFut().listen(f -> {
+            if (f.error() == null)
+                cpEndFut.complete(true);
+            else
+                cpEndFut.completeExceptionally(f.error());
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onMarkCheckpointBegin(Context ctx) {
+        // Write lock is helded. Partition pages counters has been collected 
under write lock.
+        if (stopping.getAsBoolean())
+            return;
+
+        try {
+            for (Map.Entry<Integer, Set<Integer>> e : parts.entrySet()) {
+                int grpId = e.getKey();
+                Set<Integer> grpParts = e.getValue();
+
+                GridDhtPartitionTopology top = 
cctx.cache().cacheGroup(grpId).topology();
+
+                Iterator<GridDhtLocalPartition> iter;
+
+                if (e.getValue() == null)
+                    iter = top.currentLocalPartitions().iterator();
+                else {
+                    if (grpParts.contains(INDEX_PARTITION)) {
+                        throw new IgniteCheckedException("Index partition 
cannot be included into snapshot if " +
+                            " set of cache group partitions has been 
explicitly provided [grpId=" + grpId + ']');
+                    }
+
+                    iter = F.iterator(grpParts, top::localPartition, false);
+                }
+
+                Set<Integer> owning = processed.computeIfAbsent(grpId, g -> 
new HashSet<>());
+                Set<Integer> missed = new HashSet<>();
+
+                // Iterate over partition in particular cache group
+                while (iter.hasNext()) {
+                    GridDhtLocalPartition part = iter.next();
+
+                    // Partition can be reserved.
+                    // Partition can be MOVING\RENTING states.
+                    // Index partition will be excluded if not all partition 
OWNING.
+                    // There is no data assigned to partition, thus it haven't 
been created yet.
+                    if (part.state() == GridDhtPartitionState.OWNING)
+                        owning.add(part.id());
+                    else
+                        missed.add(part.id());
+                }
+
+                if (grpParts == null) {
 
 Review comment:
   I think there should be `grpParts != null`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

Reply via email to