diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 22b07cf579a68..2a7547d59fb42 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -419,6 +419,7 @@ public void testGetAllView() throws Exception {
new TreeSet<>(Arrays.asList(
"SYS.METRICS",
"SYS.SERVICES",
+ "SYS.SNAPSHOT",
"SYS.CACHE_GROUPS",
"SYS.CACHES",
"SYS.TASKS",
@@ -896,6 +897,13 @@ public void testGetAllColumns() throws Exception {
"SYS.SERVICES.STATICALLY_CONFIGURED.null",
"SYS.SERVICES.SERVICE_ID.null",
"SYS.SERVICES.TOPOLOGY_SNAPSHOT.null",
+ "SYS.SNAPSHOT.BASELINE_NODES.null",
+ "SYS.SNAPSHOT.CACHE_GROUPS.null",
+ "SYS.SNAPSHOT.CONSISTENT_ID.null",
+ "SYS.SNAPSHOT.INCREMENT_INDEX.null",
+ "SYS.SNAPSHOT.NAME.null",
+ "SYS.SNAPSHOT.SNAPSHOT_RECORD_SEGMENT.null",
+ "SYS.SNAPSHOT.TYPE.null",
"SYS.TASKS.AFFINITY_CACHE_NAME.null",
"SYS.TASKS.INTERNAL.null",
"SYS.TASKS.END_TIME.null",
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 2f6cc9f49887c..e66d742ba17e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSnapshot.java
@@ -34,6 +34,8 @@
*
Concurrent updates from {@link IgniteDataStreamer} with default {@link IgniteDataStreamer#allowOverwrite()}
* setting (false) into a persistent cache can cause that cache data stored inconsistent.
*
+ * You can create cache dump, also. Cache dump is consistent files consist of entry by entry cache content.
+ * This type of snapshots supports in-memory caches.
*/
public interface IgniteSnapshot {
/**
@@ -61,6 +63,16 @@ public interface IgniteSnapshot {
*/
public IgniteFuture cancelSnapshot(String name);
+ /**
+ * Creates cache groups dump.
+ * Dump is consistent entry by entry backup of cache group content.
+ * In-memory cache groups also supported.
+ *
+ * @param name Dump name.
+ * @return Future which will be completed when dump ends.
+ */
+ public IgniteFuture createDump(String name);
+
/**
* Restore cache group(s) from the snapshot.
*
diff --git a/modules/core/src/main/java/org/apache/ignite/dump/DumpConsumer.java b/modules/core/src/main/java/org/apache/ignite/dump/DumpConsumer.java
new file mode 100644
index 0000000000000..aec3a02c99789
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/dump/DumpConsumer.java
@@ -0,0 +1,83 @@
+/*
+ * 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.dump;
+
+import java.util.Iterator;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cdc.TypeMapping;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.Dump;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * Consumer of {@link Dump}.
+ * This consumer will receive all {@link DumpEntry} stored in cache dump during {@code IgniteDumpReader} application invocation.
+ * The lifecycle of the consumer is the following:
+ *
+ * Start of the consumer {@link #start()}.
+ * Stop of the consumer {@link #stop()}.
+ *
+ *
+ */
+@IgniteExperimental
+public interface DumpConsumer {
+ /**
+ * Starts the consumer.
+ */
+ void start();
+
+ /**
+ * Handles type mappings.
+ * @param mappings Mappings iterator.
+ */
+ void onMappings(Iterator mappings);
+
+ /**
+ * Handles binary types.
+ * @param types Binary types iterator.
+ */
+ void onTypes(Iterator types);
+
+ /**
+ * Handles cache configs.
+ * Note, there can be several copies of cache config in the dump.
+ * This can happen if dump contains data from several nodes.
+ * @param caches Stored cache data.
+ */
+ void onCacheConfigs(Iterator caches);
+
+ /**
+ * Handles cache data.
+ * This method can be invoced by several threads concurrently.
+ * Note, there can be several copies of group partition in the dump.
+ * This can happen if dump contains data from several nodes.
+ * In this case callback will be invoked several time for the same pair of [grp, part] values.
+ *
+ * @param grp Group id.
+ * @param part Partition.
+ * @param data Cache data iterator.
+ * @see DumpReaderConfiguration#threadCount()
+ */
+ void onPartition(int grp, int part, Iterator data);
+
+ /**
+ * Stops the consumer.
+ * This method can be invoked only after {@link #start()}.
+ */
+ void stop();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/dump/DumpEntry.java b/modules/core/src/main/java/org/apache/ignite/dump/DumpEntry.java
new file mode 100644
index 0000000000000..ca34f3cbacea9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/dump/DumpEntry.java
@@ -0,0 +1,44 @@
+/*
+ * 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.dump;
+
+import java.util.Iterator;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.Dump;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * Single cache entry from dump.
+ *
+ * @see Dump#iterator(String, int, int)
+ * @see DumpConsumer#onPartition(int, int, Iterator)
+ * @see org.apache.ignite.IgniteSnapshot#createDump(String)
+ */
+@IgniteExperimental
+public interface DumpEntry {
+ /** @return Cache id. */
+ public int cacheId();
+
+ /** @return Expiration time. */
+ public long expireTime();
+
+ /** @return Key. */
+ public Object key();
+
+ /** @return Value. */
+ public Object value();
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/dump/DumpReader.java b/modules/core/src/main/java/org/apache/ignite/dump/DumpReader.java
new file mode 100644
index 0000000000000..07b9bf8035ee3
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/dump/DumpReader.java
@@ -0,0 +1,212 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.GridLoggerProxy;
+import org.apache.ignite.internal.binary.BinaryUtils;
+import org.apache.ignite.internal.cdc.CdcMain;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadata;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.Dump;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.Dump.DumpedPartitionIterator;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteExperimental;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_MARSHALLER_PATH;
+import static org.apache.ignite.internal.IgniteKernal.NL;
+import static org.apache.ignite.internal.IgniteKernal.SITE;
+import static org.apache.ignite.internal.IgniteVersionUtils.ACK_VER_STR;
+import static org.apache.ignite.internal.IgniteVersionUtils.COPYRIGHT;
+
+/**
+ * Dump Reader application.
+ * The application runs independently of Ignite node process and provides the ability to the {@link DumpConsumer} to consume
+ * all data stored in cache dump ({@link Dump})
+ */
+@IgniteExperimental
+public class DumpReader implements Runnable {
+ /** Configuration. */
+ private final DumpReaderConfiguration cfg;
+
+ /** Log. */
+ private final IgniteLogger log;
+
+ /**
+ * @param cfg Dump reader configuration.
+ * @param log Logger.
+ */
+ public DumpReader(DumpReaderConfiguration cfg, IgniteLogger log) {
+ this.cfg = cfg;
+ this.log = log.getLogger(DumpReader.class);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void run() {
+ ackAsciiLogo();
+
+ try (Dump dump = new Dump(cfg.dumpRoot(), cfg.keepBinary(), false, log)) {
+ DumpConsumer cnsmr = cfg.consumer();
+
+ cnsmr.start();
+
+ try {
+ File[] files = new File(cfg.dumpRoot(), DFLT_MARSHALLER_PATH).listFiles(BinaryUtils::notTmpFile);
+
+ if (files != null)
+ cnsmr.onMappings(CdcMain.typeMappingIterator(files, tm -> true));
+
+ cnsmr.onTypes(dump.types());
+
+ Map> grpToNodes = new HashMap<>();
+
+ for (SnapshotMetadata meta : dump.metadata()) {
+ for (Integer grp : meta.cacheGroupIds())
+ grpToNodes.computeIfAbsent(grp, key -> new ArrayList<>()).add(meta.folderName());
+ }
+
+ cnsmr.onCacheConfigs(grpToNodes.entrySet().stream()
+ .flatMap(e -> dump.configs(F.first(e.getValue()), e.getKey()).stream())
+ .iterator());
+
+ ExecutorService execSvc = cfg.threadCount() > 1 ? Executors.newFixedThreadPool(cfg.threadCount()) : null;
+
+ AtomicBoolean skip = new AtomicBoolean(false);
+
+ for (Map.Entry> e : grpToNodes.entrySet()) {
+ int grp = e.getKey();
+
+ for (String node : e.getValue()) {
+ for (int part : dump.partitions(node, grp)) {
+ Runnable consumePart = () -> {
+ if (skip.get()) {
+ if (log.isDebugEnabled()) {
+ log.debug("Skip partition due to previous error [node=" + node + ", grp=" + grp +
+ ", part=" + part + ']');
+ }
+
+ return;
+ }
+
+ try (DumpedPartitionIterator iter = dump.iterator(node, grp, part)) {
+ if (log.isDebugEnabled()) {
+ log.debug("Consuming partition [node=" + node + ", grp=" + grp +
+ ", part=" + part + ']');
+ }
+
+ cnsmr.onPartition(grp, part, iter);
+ }
+ catch (Exception ex) {
+ skip.set(cfg.failFast());
+
+ log.error("Error consuming partition [node=" + node + ", grp=" + grp +
+ ", part=" + part + ']', ex);
+
+ throw new IgniteException(ex);
+ }
+ };
+
+ if (cfg.threadCount() > 1)
+ execSvc.submit(consumePart);
+ else
+ consumePart.run();
+ }
+ }
+ }
+
+ if (cfg.threadCount() > 1) {
+ execSvc.shutdown();
+
+ boolean res = execSvc.awaitTermination(cfg.timeout().toMillis(), MILLISECONDS);
+
+ if (!res) {
+ log.warning("Dump processing tasks not finished after timeout. Cancelling");
+
+ execSvc.shutdownNow();
+ }
+ }
+ }
+ finally {
+ cnsmr.stop();
+ }
+ }
+ catch (Exception e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /** */
+ private void ackAsciiLogo() {
+ String ver = "ver. " + ACK_VER_STR;
+
+ if (log.isInfoEnabled()) {
+ log.info(NL + NL +
+ ">>> __________ ________________ ___ __ ____ ______ ___ _______ ___ _______" + NL +
+ ">>> / _/ ___/ |/ / _/_ __/ __/ / _ \\/ / / / |/ / _ \\ / _ \\/ __/ _ | / _ \\/ __/ _ \\" + NL +
+ ">>> _/ // (_ / // / / / / _/ / // / /_/ / /|_/ / ___/ / , _/ _// __ |/ // / _// , _/" + NL +
+ ">>> /___/\\___/_/|_/___/ /_/ /___/ /____/\\____/_/ /_/_/ /_/|_/___/_/ |_/____/___/_/|_|" + NL +
+ ">>> " + NL +
+ ">>> " + ver + NL +
+ ">>> " + COPYRIGHT + NL +
+ ">>> " + NL +
+ ">>> Ignite documentation: " + "http://" + SITE + NL +
+ ">>> ConsistentId: " + cfg.dumpRoot() + NL +
+ ">>> Consumer: " + U.toStringSafe(cfg.consumer())
+ );
+ }
+
+ if (log.isQuiet()) {
+ U.quiet(false,
+ " __________ ________________ ___ __ ____ ______ ___ _______ ___ _______",
+ " / _/ ___/ |/ / _/_ __/ __/ / _ \\/ / / / |/ / _ \\ / _ \\/ __/ _ | / _ \\/ __/ _ \\",
+ " _/ // (_ / // / / / / _/ / // / /_/ / /|_/ / ___/ / , _/ _// __ |/ // / _// , _/",
+ "/___/\\___/_/|_/___/ /_/ /___/ /____/\\____/_/ /_/_/ /_/|_/___/_/ |_/____/___/_/|_|",
+ "",
+ ver,
+ COPYRIGHT,
+ "",
+ "Ignite documentation: " + "http://" + SITE,
+ "Dump: " + cfg.dumpRoot(),
+ "Consumer: " + U.toStringSafe(cfg.consumer()),
+ "",
+ "Quiet mode.");
+
+ String fileName = log.fileName();
+
+ if (fileName != null)
+ U.quiet(false, " ^-- Logging to file '" + fileName + '\'');
+
+ if (log instanceof GridLoggerProxy)
+ U.quiet(false, " ^-- Logging by '" + ((GridLoggerProxy)log).getLoggerInfo() + '\'');
+
+ U.quiet(false,
+ " ^-- To see **FULL** console log here add -DIGNITE_QUIET=false or \"-v\" to ignite-cdc.{sh|bat}",
+ "");
+ }
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/dump/DumpReaderConfiguration.java b/modules/core/src/main/java/org/apache/ignite/dump/DumpReaderConfiguration.java
new file mode 100644
index 0000000000000..c967136fe81a7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/dump/DumpReaderConfiguration.java
@@ -0,0 +1,112 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.time.Duration;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.lang.IgniteExperimental;
+
+/**
+ * Configuration class of {@link DumpReader}.
+ *
+ * @see DumpReader
+ * @see DumpConsumer
+ */
+@IgniteExperimental
+public class DumpReaderConfiguration {
+ /** Default timeout. */
+ public static final Duration DFLT_TIMEOUT = Duration.ofDays(7);
+
+ /** Default thread count. */
+ public static final int DFLT_THREAD_CNT = 1;
+
+ /** Root dump directory. */
+ private final File dir;
+
+ /** Dump consumer. */
+ private final DumpConsumer cnsmr;
+
+ /** Count of threads to consume dumped partitions. */
+ private final int thCnt;
+
+ /** Timeout of dump reader. 1 week, by default. */
+ private final Duration timeout;
+
+ /** Stop processing partitions if consumer fail to process one. */
+ private final boolean failFast;
+
+ /** If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}. */
+ private final boolean keepBinary;
+
+ /**
+ * @param dir Root dump directory.
+ * @param cnsmr Dump consumer.
+ */
+ public DumpReaderConfiguration(File dir, DumpConsumer cnsmr) {
+ this(dir, cnsmr, DFLT_THREAD_CNT, DFLT_TIMEOUT, true, true);
+ }
+
+ /**
+ * @param dir Root dump directory.
+ * @param cnsmr Dump consumer.
+ * @param thCnt Count of threads to consume dumped partitions.
+ * @param timeout Timeout of dump reader invocation.
+ * @param failFast Stop processing partitions if consumer fail to process one.
+ * @param keepBinary If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}.
+ */
+ public DumpReaderConfiguration(File dir, DumpConsumer cnsmr, int thCnt, Duration timeout, boolean failFast, boolean keepBinary) {
+ this.dir = dir;
+ this.cnsmr = cnsmr;
+ this.thCnt = thCnt;
+ this.timeout = timeout;
+ this.failFast = failFast;
+ this.keepBinary = keepBinary;
+ }
+
+ /** @return Root dump directiory. */
+ public File dumpRoot() {
+ return dir;
+ }
+
+ /** @return Dump consumer instance. */
+ public DumpConsumer consumer() {
+ return cnsmr;
+ }
+
+ /** @return Count of threads to consume dumped partitions. */
+ public int threadCount() {
+ return thCnt;
+ }
+
+ /** @return Timeout of dump reader invocation. */
+ public Duration timeout() {
+ return timeout;
+ }
+
+ /** @return {@code True} if stop processing after first consumer error. */
+ public boolean failFast() {
+ return failFast;
+ }
+
+ /** @return If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}. */
+ public boolean keepBinary() {
+ return keepBinary;
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java
index 1e73e75be2204..2e5914d58d273 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcMain.java
@@ -31,6 +31,7 @@
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
import java.util.stream.Stream;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
@@ -44,7 +45,6 @@
import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.GridComponent;
import org.apache.ignite.internal.GridLoggerProxy;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.MarshallerContextImpl;
@@ -82,6 +82,8 @@
import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_RECORD_V2;
import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_SEGMENT_FILE_FILTER;
import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.segmentIndex;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.closeAllComponents;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.startAllComponents;
import static org.apache.ignite.internal.processors.metric.impl.MetricUtils.metricName;
/**
@@ -328,8 +330,7 @@ public void runX() throws Exception {
}
}
finally {
- for (GridComponent comp : kctx)
- comp.stop(false);
+ closeAllComponents(kctx);
if (log.isInfoEnabled())
log.info("Ignite Change Data Capture Application stopped.");
@@ -370,8 +371,7 @@ private StandaloneGridKernalContext startStandaloneKernal() throws IgniteChecked
kctx.resource().setSpringContext(ctx);
- for (GridComponent comp : kctx)
- comp.start();
+ startAllComponents(kctx);
mreg = kctx.metric().registry("cdc");
@@ -647,27 +647,10 @@ private void updateMappings() {
if (files == null)
return;
- Iterator changedMappings = Arrays.stream(files)
- .map(f -> {
- String fileName = f.getName();
-
- int typeId = BinaryUtils.mappedTypeId(fileName);
- byte platformId = BinaryUtils.mappedFilePlatformId(fileName);
-
- T2 state = new T2<>(typeId, platformId);
-
- if (mappingsState.contains(state))
- return null;
-
- mappingsState.add(state);
-
- return (TypeMapping)new TypeMappingImpl(
- typeId,
- BinaryUtils.readMapping(f),
- platformId == 0 ? PlatformType.JAVA : PlatformType.DOTNET);
- })
- .filter(Objects::nonNull)
- .iterator();
+ Iterator changedMappings = typeMappingIterator(
+ files,
+ tm -> mappingsState.add(new T2<>(tm.typeId(), (byte)tm.platformType().ordinal()))
+ );
if (!changedMappings.hasNext())
return;
@@ -867,4 +850,27 @@ private void ackAsciiLogo() {
public static String cdcInstanceName(String igniteInstanceName) {
return "cdc-" + igniteInstanceName;
}
+
+ /**
+ * @param files Mapping files.
+ * @param filter Filter.
+ * @return Type mapping iterator.
+ */
+ public static Iterator typeMappingIterator(File[] files, Predicate filter) {
+ return Arrays.stream(files)
+ .map(f -> {
+ String fileName = f.getName();
+
+ int typeId = BinaryUtils.mappedTypeId(fileName);
+ byte platformId = BinaryUtils.mappedFilePlatformId(fileName);
+
+ return (TypeMapping)new TypeMappingImpl(
+ typeId,
+ BinaryUtils.readMapping(f),
+ platformId == 0 ? PlatformType.JAVA : PlatformType.DOTNET);
+ })
+ .filter(filter)
+ .filter(Objects::nonNull)
+ .iterator();
+ }
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcUtils.java
new file mode 100644
index 0000000000000..c53fc7fed9837
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cdc/CdcUtils.java
@@ -0,0 +1,60 @@
+/*
+ * 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.cdc;
+
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cdc.TypeMapping;
+import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.binary.BinaryMetadata;
+
+/**
+ * Methods to reuse various CDC like utilities.
+ */
+public class CdcUtils {
+ /**
+ * Register {@code meta}.
+ *
+ * @param ctx Binary context.
+ * @param log Logger.
+ * @param meta Binary metadata to register.
+ */
+ public static void registerBinaryMeta(BinaryContext ctx, IgniteLogger log, BinaryMetadata meta) {
+ ctx.updateMetadata(meta.typeId(), meta, false);
+
+ if (log.isInfoEnabled())
+ log.info("BinaryMeta [meta=" + meta + ']');
+ }
+
+ /**
+ * Register {@code mapping}.
+ *
+ * @param ctx Binary context.
+ * @param log Logger.
+ * @param mapping Type mapping to register.
+ */
+ public static void registerMapping(BinaryContext ctx, IgniteLogger log, TypeMapping mapping) {
+ assert mapping.platformType().ordinal() <= Byte.MAX_VALUE;
+
+ byte platformType = (byte)mapping.platformType().ordinal();
+
+ ctx.registerUserClassName(mapping.typeId(), mapping.typeName(), false, false, platformType);
+
+ if (log.isInfoEnabled())
+ log.info("Mapping [mapping=" + mapping + ']');
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
index 216e7332fbd26..f69f579084f69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/UnwrapDataEntry.java
@@ -71,18 +71,7 @@ public UnwrapDataEntry(
/** {@inheritDoc} */
@Override public Object unwrappedKey() {
try {
- if (keepBinary && key instanceof BinaryObject)
- return key;
-
- Object unwrapped = key.value(cacheObjValCtx, false);
-
- if (unwrapped instanceof BinaryObject) {
- if (keepBinary)
- return unwrapped;
- unwrapped = ((BinaryObject)unwrapped).deserialize();
- }
-
- return unwrapped;
+ return unwrapKey(key, keepBinary, cacheObjValCtx);
}
catch (Exception e) {
cacheObjValCtx.kernalContext().log(UnwrapDataEntry.class)
@@ -95,13 +84,7 @@ public UnwrapDataEntry(
/** {@inheritDoc} */
@Override public Object unwrappedValue() {
try {
- if (val == null)
- return null;
-
- if (keepBinary && val instanceof BinaryObject)
- return val;
-
- return val.value(cacheObjValCtx, false);
+ return unwrapValue(val, keepBinary, cacheObjValCtx);
}
catch (Exception e) {
cacheObjValCtx.kernalContext().log(UnwrapDataEntry.class)
@@ -110,6 +93,33 @@ public UnwrapDataEntry(
}
}
+ /** */
+ public static Object unwrapKey(KeyCacheObject key, boolean keepBinary, CacheObjectValueContext cacheObjValCtx) {
+ if (keepBinary && key instanceof BinaryObject)
+ return key;
+
+ Object unwrapped = key.value(cacheObjValCtx, false);
+
+ if (unwrapped instanceof BinaryObject) {
+ if (keepBinary)
+ return unwrapped;
+ unwrapped = ((BinaryObject)unwrapped).deserialize();
+ }
+
+ return unwrapped;
+ }
+
+ /** */
+ public static Object unwrapValue(CacheObject val, boolean keepBinary, CacheObjectValueContext cacheObjValCtx) {
+ if (val == null)
+ return null;
+
+ if (keepBinary && val instanceof BinaryObject)
+ return val;
+
+ return val.value(cacheObjValCtx, false);
+ }
+
/** {@inheritDoc} */
@Override public String toString() {
SB sb = new SB();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 3462e0b3bbe20..2c15eee871263 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -79,6 +79,7 @@
import org.apache.ignite.internal.processors.cache.dr.GridCacheDrManager;
import org.apache.ignite.internal.processors.cache.jta.CacheJtaManagerAdapter;
import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.DumpEntryChangeListener;
import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
import org.apache.ignite.internal.processors.cache.store.CacheStoreManager;
@@ -283,6 +284,9 @@ public class GridCacheContext implements Externalizable {
/** Recovery mode flag. */
private volatile boolean recoveryMode;
+ /** Dump callback. */
+ private volatile DumpEntryChangeListener dumpLsnr;
+
/** */
private final boolean disableTriggeringCacheInterceptorOnConflict =
Boolean.parseBoolean(System.getProperty(IGNITE_DISABLE_TRIGGERING_CACHE_INTERCEPTOR_ON_CONFLICT, "false"));
@@ -2333,6 +2337,19 @@ public AtomicReference> lastRemoveAllJobFut() {
return lastRmvAllJobFut;
}
+ /** */
+ public DumpEntryChangeListener dumpListener() {
+ return dumpLsnr;
+ }
+
+ /** */
+ public void dumpListener(DumpEntryChangeListener dumpEntryChangeLsnr) {
+ assert this.dumpLsnr == null || dumpEntryChangeLsnr == null;
+ assert cacheType == CacheType.USER;
+
+ this.dumpLsnr = dumpEntryChangeLsnr;
+ }
+
/** {@inheritDoc} */
@Override public void writeExternal(ObjectOutput out) throws IOException {
U.writeString(out, igniteInstanceName());
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index cb2de2968e2bc..ede08985d78ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -66,6 +66,7 @@
import org.apache.ignite.internal.processors.cache.persistence.CacheDataRowAdapter;
import org.apache.ignite.internal.processors.cache.persistence.DataRegion;
import org.apache.ignite.internal.processors.cache.persistence.StorageException;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.DumpEntryChangeListener;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryListener;
import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
@@ -1463,6 +1464,11 @@ else if (interceptorVal != val0)
assert ttl >= 0 : ttl;
assert expireTime >= 0 : expireTime;
+ DumpEntryChangeListener dumpLsnr = cctx.dumpListener();
+
+ if (dumpLsnr != null)
+ dumpLsnr.beforeChange(cctx, key, old, extras == null ? CU.EXPIRE_TIME_ETERNAL : extras.expireTime(), ver);
+
// Detach value before index update.
val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);
@@ -1656,6 +1662,11 @@ protected Object keyValue(boolean cpy) {
}
}
+ DumpEntryChangeListener dumpLsnr = cctx.dumpListener();
+
+ if (dumpLsnr != null)
+ dumpLsnr.beforeChange(cctx, key, old, extras == null ? CU.EXPIRE_TIME_ETERNAL : extras.expireTime(), ver);
+
removeValue();
update(null, 0, 0, newVer, true);
@@ -3599,6 +3610,11 @@ private boolean onExpired(CacheObject expiredVal, GridCacheVersion obsoleteVer)
if (mvccExtras() != null)
return false;
+ DumpEntryChangeListener dumpLsnr = cctx.dumpListener();
+
+ if (dumpLsnr != null)
+ dumpLsnr.beforeChange(cctx, key, expiredVal, extras == null ? CU.TTL_MINIMUM : extras.expireTime(), ver);
+
if (cctx.deferredDelete() && !detached() && !isInternal()) {
if (!deletedUnlocked() && !isStartVersion()) {
update(null, 0L, 0L, ver, true);
@@ -5965,6 +5981,18 @@ else if (interceptorVal != updated0) {
}
}
+ DumpEntryChangeListener dumpLsnr = cctx.dumpListener();
+
+ if (dumpLsnr != null) {
+ dumpLsnr.beforeChange(
+ cctx,
+ entry.key,
+ oldVal,
+ entry.extras == null ? CU.EXPIRE_TIME_ETERNAL : entry.extras.expireTime(),
+ entry.ver
+ );
+ }
+
updated = cctx.kernalContext().cacheObjects().prepareForCache(updated, cctx);
if (writeThrough)
@@ -6068,6 +6096,18 @@ private void remove(@Nullable GridCacheVersionConflictContext, ?> conflictCtx,
}
}
+ DumpEntryChangeListener dumpLsnr = cctx.dumpListener();
+
+ if (dumpLsnr != null) {
+ dumpLsnr.beforeChange(
+ cctx,
+ entry.key,
+ oldVal,
+ entry.extras == null ? CU.EXPIRE_TIME_ETERNAL : entry.extras.expireTime(),
+ entry.ver
+ );
+ }
+
if (writeThrough)
// Must persist inside synchronization in non-tx mode.
cctx.store().remove(null, entry.key);
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
index 6593a31c5c793..eeb9c0d896537 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridLocalConfigManager.java
@@ -540,8 +540,12 @@ private boolean inMemoryCdcCache(CacheConfiguration, ?> cfg) {
public File cacheConfigurationFile(CacheConfiguration, ?> ccfg) {
File cacheWorkDir = cacheWorkDir(ccfg);
- return ccfg.getGroupName() == null ? new File(cacheWorkDir, CACHE_DATA_FILENAME) :
- new File(cacheWorkDir, ccfg.getName() + CACHE_DATA_FILENAME);
+ return new File(cacheWorkDir, cacheDataFilename(ccfg));
+ }
+
+ /** @return Name of cache data filename. */
+ public static String cacheDataFilename(CacheConfiguration, ?> ccfg) {
+ return ccfg.getGroupName() == null ? CACHE_DATA_FILENAME : (ccfg.getName() + CACHE_DATA_FILENAME);
}
/**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
index b994c4b7d0621..db43a7a0c113a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/BinaryMetadataFileStore.java
@@ -82,35 +82,40 @@ class BinaryMetadataFileStore {
* @param log Logger.
* @param binaryMetadataFileStoreDir Path to binary metadata store configured by user, should include binary_meta
* and consistentId.
+ * @param forceEnabled If {@code true} then will write files even if persistence and CDC disabled.
*/
BinaryMetadataFileStore(
final ConcurrentMap metadataLocCache,
final GridKernalContext ctx,
final IgniteLogger log,
- final File binaryMetadataFileStoreDir
+ final File binaryMetadataFileStoreDir,
+ final boolean forceEnabled
) throws IgniteCheckedException {
this.metadataLocCache = metadataLocCache;
this.ctx = ctx;
- enabled = CU.isPersistenceEnabled(ctx.config()) || CU.isCdcEnabled(ctx.config());
+ enabled = forceEnabled || CU.isPersistenceEnabled(ctx.config()) || CU.isCdcEnabled(ctx.config());
this.log = log;
if (!enabled)
return;
- fileIOFactory = ctx.config().getDataStorageConfiguration().getFileIOFactory();
+ DataStorageConfiguration dsCfg = ctx.config().getDataStorageConfiguration();
+
+ fileIOFactory = dsCfg == null ? new DataStorageConfiguration().getFileIOFactory() : dsCfg.getFileIOFactory();
final String nodeFolderName = ctx.pdsFolderResolver().resolveFolders().folderName();
if (binaryMetadataFileStoreDir != null)
metadataDir = binaryMetadataFileStoreDir;
- else
+ else {
metadataDir = new File(U.resolveWorkDirectory(
ctx.config().getWorkDirectory(),
DataStorageConfiguration.DFLT_BINARY_METADATA_PATH,
false
), nodeFolderName);
+ }
fixLegacyFolder(nodeFolderName);
}
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 db8bb33c8b459..7ca52e6644957 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
@@ -261,7 +261,8 @@ public static File binaryWorkDir(String igniteWorkDir, String consId) {
CU.isPersistenceEnabled(ctx.config()) && binaryMetadataFileStoreDir == null ?
resolveBinaryWorkDir(ctx.config().getWorkDirectory(),
ctx.pdsFolderResolver().resolveFolders().folderName()) :
- binaryMetadataFileStoreDir);
+ binaryMetadataFileStoreDir,
+ false);
metadataFileStore.start();
}
@@ -1019,7 +1020,9 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException {
ctx,
log,
resolveBinaryWorkDir(dir.getAbsolutePath(),
- ctx.pdsFolderResolver().resolveFolders().folderName()));
+ ctx.pdsFolderResolver().resolveFolders().folderName()),
+ true
+ );
for (BinaryType type : types)
writer.mergeAndWriteMetadata(((BinaryTypeImpl)type).metadata());
@@ -1037,7 +1040,7 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException {
try {
ConcurrentMap metaCache = new ConcurrentHashMap<>();
- new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir)
+ new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir, false)
.restoreMetadata();
Collection metadata = F.viewReadOnly(metaCache.values(), BinaryMetadataHolder::metadata);
@@ -1073,7 +1076,7 @@ public BinaryMetadata binaryMetadata(int typeId) throws BinaryObjectException {
ConcurrentMap metaCache = new ConcurrentHashMap<>();
- new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir).restoreMetadata(typeId);
+ new BinaryMetadataFileStore(metaCache, ctx, log, metadataDir, false).restoreMetadata(typeId);
addMetaLocally(typeId, metaCache.get(typeId).metadata().wrap(binaryContext()), false);
}
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 af0a4975d1e78..ee2d6d87ea0bb 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
@@ -916,6 +916,15 @@ public static int partId(String partFileName) {
* @return List of cache partitions in given directory.
*/
public static List cachePartitionFiles(File cacheDir) {
+ return cachePartitionFiles(cacheDir, FILE_SUFFIX);
+ }
+
+ /**
+ * @param cacheDir Cache directory to check.
+ * @param ext File extension.
+ * @return List of cache partitions in given directory.
+ */
+ public static List cachePartitionFiles(File cacheDir, String ext) {
File[] files = cacheDir.listFiles();
if (files == null)
@@ -923,7 +932,7 @@ public static List cachePartitionFiles(File cacheDir) {
return Arrays.stream(files)
.filter(File::isFile)
- .filter(f -> f.getName().endsWith(FILE_SUFFIX))
+ .filter(f -> f.getName().endsWith(ext))
.collect(Collectors.toList());
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractCreateSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractCreateSnapshotFutureTask.java
new file mode 100644
index 0000000000000..a2de0d59e6425
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractCreateSnapshotFutureTask.java
@@ -0,0 +1,248 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
+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.persistence.checkpoint.CheckpointListener;
+import org.apache.ignite.internal.processors.marshaller.MappedName;
+import org.apache.ignite.internal.util.lang.IgniteThrowableRunner;
+import org.apache.ignite.internal.util.typedef.F;
+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;
+
+/** */
+public abstract class AbstractCreateSnapshotFutureTask extends AbstractSnapshotFutureTask {
+ /**
+ * Cache group and corresponding partitions collected under the PME lock.
+ * For full snapshot additional checkpoint write lock required.
+ * @see SnapshotFutureTask#onMarkCheckpointBegin(CheckpointListener.Context)
+ */
+ protected final Map> processed = new HashMap<>();
+
+ /** Future which will be completed when task requested to be closed. Will be executed on system pool. */
+ protected volatile CompletableFuture closeFut;
+
+ /**
+ * @param cctx Shared context.
+ * @param srcNodeId Node id which cause snapshot task creation.
+ * @param reqId Snapshot operation request ID.
+ * @param snpName Snapshot name.
+ * @param snpSndr Factory which produces snapshot sender instance.
+ * @param parts Partitions to be processed.
+ */
+ protected AbstractCreateSnapshotFutureTask(
+ GridCacheSharedContext, ?> cctx,
+ UUID srcNodeId,
+ UUID reqId,
+ String snpName,
+ SnapshotSender snpSndr,
+ Map> parts
+ ) {
+ super(cctx, srcNodeId, reqId, snpName, snpSndr, parts);
+ }
+
+ /** */
+ protected abstract List> saveCacheConfigs();
+
+ /** */
+ protected abstract List> saveGroup(int grpId, Set grpParts) throws IgniteCheckedException;
+
+ /** {@inheritDoc} */
+ @Override public boolean cancel() {
+ super.cancel();
+
+ try {
+ closeAsync().get();
+ }
+ catch (InterruptedException | ExecutionException e) {
+ U.error(log, "SnapshotFutureTask cancellation failed", e);
+
+ return false;
+ }
+
+ return true;
+ }
+
+ /** @return Future which will be completed when operations truly stopped. */
+ protected abstract CompletableFuture closeAsync();
+
+ /**
+ * @return {@code true} if current task requested to be stopped.
+ */
+ protected boolean stopping() {
+ return err.get() != null;
+ }
+
+ /** */
+ protected void processPartitions() throws IgniteCheckedException {
+ for (Map.Entry> e : parts.entrySet()) {
+ int grpId = e.getKey();
+ Set grpParts = e.getValue();
+
+ CacheGroupContext gctx = cctx.cache().cacheGroup(grpId);
+
+ Iterator iter;
+
+ if (grpParts == null)
+ iter = gctx.topology().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, gctx.topology()::localPartition, false);
+ }
+
+ Set owning = new HashSet<>();
+ Set missed = new HashSet<>();
+
+ // Iterate over partitions in particular cache group.
+ while (iter.hasNext()) {
+ GridDhtLocalPartition part = iter.next();
+
+ // Partition can be in 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());
+ }
+
+ boolean affNode = gctx.nodeFilter() == null || gctx.nodeFilter().apply(cctx.localNode());
+
+ if (grpParts != null) {
+ // Partition has been provided for cache group, but some of them are not in OWNING state.
+ // Exit with an error.
+ if (!missed.isEmpty()) {
+ throw new IgniteCheckedException("Snapshot operation cancelled due to " +
+ "not all of requested partitions has OWNING state on local node [grpId=" + grpId +
+ ", missed=" + S.toStringSortedDistinct(missed) + ']');
+ }
+ }
+ else {
+ // Partitions have not been provided for snapshot task and all partitions have
+ // OWNING state, so index partition must be included into snapshot.
+ if (!missed.isEmpty()) {
+ log.warning("All local cache group partitions in OWNING state have been included into a snapshot. " +
+ "Partitions which have different states skipped. Index partitions has also been skipped " +
+ "[snpName=" + snpName + ", grpId=" + grpId + ", missed=" + S.toStringSortedDistinct(missed) + ']');
+ }
+ else if (affNode && missed.isEmpty() && cctx.kernalContext().query().moduleEnabled())
+ owning.add(INDEX_PARTITION);
+ }
+
+ processed.put(grpId, owning);
+ }
+ }
+
+ /** Starts async execution of all tasks required to create snapshot. */
+ protected void saveSnapshotData() {
+ try {
+ // Submit all tasks for partitions and deltas processing.
+ List> futs = new ArrayList<>();
+
+ Collection binTypesCopy = cctx.kernalContext()
+ .cacheObjects()
+ .metadata();
+
+ List> mappingsCopy = cctx.kernalContext()
+ .marshallerContext()
+ .getCachedMappings();
+
+ // Process binary meta.
+ futs.add(runAsync(() -> snpSndr.sendBinaryMeta(binTypesCopy)));
+ // Process marshaller meta.
+ futs.add(runAsync(() -> snpSndr.sendMarshallerMeta(mappingsCopy)));
+ futs.addAll(saveCacheConfigs());
+
+ for (Map.Entry> grpParts : processed.entrySet())
+ futs.addAll(saveGroup(grpParts.getKey(), grpParts.getValue()));
+
+ int futsSize = futs.size();
+
+ CompletableFuture.allOf(futs.toArray(new CompletableFuture[futsSize])).whenComplete((res, t) -> {
+ assert t == null : "Exception must never be thrown since a wrapper is used " +
+ "for each snapshot task: " + t;
+
+ closeAsync();
+ });
+ }
+ catch (IgniteCheckedException e) {
+ acceptException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void acceptException(Throwable th) {
+ if (th == null)
+ return;
+
+ if (!(th instanceof IgniteFutureCancelledCheckedException))
+ U.error(log, "Snapshot task has accepted exception to stop", th);
+
+ if (err.compareAndSet(null, th))
+ closeAsync();
+ }
+
+ /**
+ * @param exec Runnable task to execute.
+ * @return Wrapped task.
+ */
+ Runnable wrapExceptionIfStarted(IgniteThrowableRunner exec) {
+ return () -> {
+ if (stopping())
+ return;
+
+ try {
+ exec.run();
+ }
+ catch (Throwable t) {
+ acceptException(t);
+ }
+ };
+ }
+
+ /** */
+ protected CompletableFuture runAsync(IgniteThrowableRunner task) {
+ return CompletableFuture.runAsync(
+ wrapExceptionIfStarted(task),
+ snpSndr.executor()
+ );
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
index 7ccc2f3944c7c..12d0f755a6aaf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/AbstractSnapshotFutureTask.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.processors.cache.persistence.snapshot;
-import java.io.File;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
@@ -25,7 +24,6 @@
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.typedef.internal.S;
@@ -49,12 +47,6 @@ abstract class AbstractSnapshotFutureTask extends GridFutureAdapter {
/** Unique identifier of snapshot process. */
protected final String snpName;
- /** Snapshot working directory on file system. */
- protected final File tmpSnpWorkDir;
-
- /** IO factory which will be used for creating snapshot delta-writers. */
- protected final FileIOFactory ioFactory;
-
/** Snapshot data sender. */
@GridToStringExclude
protected final SnapshotSender snpSndr;
@@ -70,8 +62,6 @@ abstract class AbstractSnapshotFutureTask extends GridFutureAdapter {
* @param srcNodeId Node id which cause snapshot task creation.
* @param reqId Snapshot operation request ID.
* @param snpName Unique identifier of snapshot process.
- * @param tmpWorkDir Working directory for intermediate snapshot results.
- * @param ioFactory Factory to working with snapshot files.
* @param snpSndr Factory which produces snapshot receiver instance.
* @param parts Partition to be processed.
*/
@@ -80,8 +70,6 @@ protected AbstractSnapshotFutureTask(
UUID srcNodeId,
UUID reqId,
String snpName,
- File tmpWorkDir,
- FileIOFactory ioFactory,
SnapshotSender snpSndr,
Map> parts
) {
@@ -90,12 +78,10 @@ protected AbstractSnapshotFutureTask(
assert snpSndr.executor() != null : "Executor service must be not null.";
this.cctx = cctx;
- this.log = cctx.logger(AbstractSnapshotFutureTask.class);
+ this.log = cctx.logger(this.getClass());
this.srcNodeId = srcNodeId;
this.reqId = reqId;
this.snpName = snpName;
- this.tmpSnpWorkDir = new File(tmpWorkDir, snpName);
- this.ioFactory = ioFactory;
this.snpSndr = snpSndr;
this.parts = 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 ff7bc60d947b1..45a47bdd93c9a 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
@@ -141,6 +141,7 @@
import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadOnlyMetastorage;
import org.apache.ignite.internal.processors.cache.persistence.metastorage.ReadWriteMetastorage;
import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPageIO;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.DataPagePayload;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
@@ -253,7 +254,8 @@
*
* These major actions available:
*
- * Create snapshot of the whole cluster cache groups by triggering PME to achieve consistency.
+ * Create snapshot of the whole persistent cluster cache groups by triggering PME to achieve consistency.
+ * Create cache dump - snapshot of cluster cache groups including in-memory.
* Create incremental snapshot using lightweight, non-blocking Consistent Cut algorithm.
*
*/
@@ -311,7 +313,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
public static final int SNAPSHOT_LIMITED_TRANSFER_BLOCK_SIZE_BYTES = 64 * 1024;
/** Metastorage key to save currently running snapshot directory path. */
- private static final String SNP_RUNNING_DIR_KEY = "snapshot-running-dir";
+ public static final String SNP_RUNNING_DIR_KEY = "snapshot-running-dir";
/** Prefix for meta store records which means that incremental snapshot creation is disabled for a cache group. */
private static final String INC_SNP_DISABLED_KEY_PREFIX = "grp-inc-snp-disabled-";
@@ -359,6 +361,9 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
/** Pattern for incremental snapshot directory names. */
public static final Pattern INC_SNP_NAME_PATTERN = U.fixedLengthNumberNamePattern(null);
+ /** Lock file for dump directory. */
+ public static final String DUMP_LOCK = "dump.lock";
+
/**
* Local buffer to perform copy-on-write operations with pages for {@code SnapshotFutureTask.PageStoreSerialWriter}s.
* It is important to have only one buffer per thread (instead of creating each buffer per
@@ -416,7 +421,7 @@ public class IgniteSnapshotManager extends GridCacheSharedManagerAdapter
private volatile FileIOFactory ioFactory = new RandomAccessFileIOFactory();
/** File store manager to create page store for restore. */
- private volatile FilePageStoreManager storeMgr;
+ private volatile @Nullable FilePageStoreManager storeMgr;
/** File store manager to create page store for restore. */
private volatile GridLocalConfigManager locCfgMgr;
@@ -525,18 +530,13 @@ public static String partDeltaFileName(int partId) {
if (ctx.clientNode())
return;
- if (!CU.isPersistenceEnabled(ctx.config()))
- return;
-
- assert cctx.pageStore() instanceof FilePageStoreManager;
-
storeMgr = (FilePageStoreManager)cctx.pageStore();
locCfgMgr = cctx.cache().configManager();
pdsSettings = cctx.kernalContext().pdsFolderResolver().resolveFolders();
locSnpDir = resolveSnapshotWorkDirectory(ctx.config());
- tmpWorkDir = U.resolveWorkDirectory(storeMgr.workDir().getAbsolutePath(), DFLT_SNAPSHOT_TMP_DIR, true);
+ tmpWorkDir = U.resolveWorkDirectory(pdsSettings.persistentStoreNodePath().getAbsolutePath(), DFLT_SNAPSHOT_TMP_DIR, true);
U.ensureDirectory(locSnpDir, "snapshot work directory", log);
U.ensureDirectory(tmpWorkDir, "temp directory for snapshot creation", log);
@@ -690,6 +690,20 @@ public static String partDeltaFileName(int partId) {
return views;
})),
Function.identity());
+
+ Arrays.stream(locSnpDir.listFiles())
+ .filter(File::isDirectory)
+ .map(dumpDir ->
+ Paths.get(dumpDir.getAbsolutePath(), DB_DEFAULT_FOLDER, pdsSettings.folderName(), DUMP_LOCK).toFile())
+ .filter(File::exists)
+ .map(File::getParentFile)
+ .forEach(lockedDumpDir -> {
+ log.warning("Found locked dump dir. " +
+ "This means, dump creation not finished prior to node fail. " +
+ "Directory will be deleted: " + lockedDumpDir);
+
+ U.delete(lockedDumpDir);
+ });
}
/** {@inheritDoc} */
@@ -740,19 +754,21 @@ public static String partDeltaFileName(int partId) {
/**
* @param snpDir Snapshot dir.
- * @param folderName Local node folder name (see {@link U#maskForFileName} with consistent id).
+ * @param pdsSettings PDS settings.
*/
- public void deleteSnapshot(File snpDir, String folderName) {
+ public void deleteSnapshot(File snpDir, PdsFolderSettings> pdsSettings) {
if (!snpDir.exists())
return;
if (!snpDir.isDirectory())
return;
+ String folderName = pdsSettings.folderName();
+
try {
File binDir = binaryWorkDir(snpDir.getAbsolutePath(), folderName);
File nodeDbDir = new File(snpDir.getAbsolutePath(), databaseRelativePath(folderName));
- File smf = new File(snpDir, snapshotMetaFileName(folderName));
+ File smf = new File(snpDir, snapshotMetaFileName(U.maskForFileName(pdsSettings.consistentId().toString())));
U.delete(binDir);
U.delete(nodeDbDir);
@@ -1027,8 +1043,6 @@ private IgniteInternalFuture initLocalIncrementalSnap
meta,
req.snapshotPath(),
req.incrementIndex(),
- tmpWorkDir,
- ioFactory,
lowPtr,
markWalFut
)).chain(fut -> {
@@ -1150,6 +1164,7 @@ private IgniteInternalFuture initLocalFullSnapshot(
req.requestId(),
parts,
withMetaStorage,
+ req.dump(),
locSndrFactory.apply(req.snapshotName(), req.snapshotPath()));
if (withMetaStorage && task0 instanceof SnapshotFutureTask) {
@@ -1184,7 +1199,8 @@ private IgniteInternalFuture initLocalFullSnapshot(
res.parts(),
res.snapshotPointer(),
cctx.gridConfig().getEncryptionSpi().masterKeyDigest(),
- req.onlyPrimary()
+ req.onlyPrimary(),
+ req.dump()
);
SnapshotHandlerContext ctx = new SnapshotHandlerContext(meta, req.groups(), cctx.localNode(), snpDir,
@@ -1382,7 +1398,7 @@ private IgniteInternalFuture initLocalSnapshotEndStag
if (req.incremental())
U.delete(incrementalSnapshotLocalDir(req.snapshotName(), req.snapshotPath(), req.incrementIndex()));
else
- deleteSnapshot(snapshotLocalDir(req.snapshotName(), req.snapshotPath()), pdsSettings.folderName());
+ deleteSnapshot(snapshotLocalDir(req.snapshotName(), req.snapshotPath()), pdsSettings);
}
else if (!F.isEmpty(req.warnings())) {
// Pass the warnings further to the next stage for the case when snapshot started from not coordinator.
@@ -1394,12 +1410,16 @@ else if (!F.isEmpty(req.warnings())) {
storeWarnings(snpReq);
}
- removeLastMetaStorageKey();
+ if (req.dump())
+ removeDumpLock(req.snapshotName());
+ else {
+ removeLastMetaStorageKey();
- if (req.error() == null) {
- Collection grpIds = req.groups().stream().map(CU::cacheId).collect(Collectors.toList());
+ if (req.error() == null) {
+ Collection grpIds = req.groups().stream().map(CU::cacheId).collect(Collectors.toList());
- enableIncrementalSnapshotsCreation(grpIds);
+ enableIncrementalSnapshotsCreation(grpIds);
+ }
}
}
catch (Exception e) {
@@ -1755,6 +1775,11 @@ private boolean cancelLocalSnapshotTask0(Function,
return new IgniteFutureImpl<>(cancelSnapshot0(name));
}
+ /** {@inheritDoc} */
+ @Override public IgniteFuture createDump(String name) {
+ return createSnapshot(name, null, false, false, true);
+ }
+
/**
* @param name Snapshot name.
*
@@ -2139,10 +2164,31 @@ public IgniteFutureImpl createSnapshot(
@Nullable String snpPath,
boolean incremental,
boolean onlyPrimary
+ ) {
+ return createSnapshot(name, snpPath, incremental, onlyPrimary, false);
+ }
+
+ /**
+ * Create a consistent copy of all persistence cache groups from the whole cluster.
+ *
+ * @param name Snapshot unique name which satisfies the following name pattern [a-zA-Z0-9_].
+ * @param snpPath Snapshot directory path.
+ * @param incremental Incremental snapshot flag.
+ * @param onlyPrimary If {@code true} snapshot only primary copies of partitions.
+ * @param dump If {@code true} cache dump must be created.
+ * @return Future which will be completed when a process ends.
+ */
+ public IgniteFutureImpl createSnapshot(
+ String name,
+ @Nullable String snpPath,
+ boolean incremental,
+ boolean onlyPrimary,
+ boolean dump
) {
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(!(incremental && onlyPrimary), "Only primary not supported for incremental snapshots");
+ A.ensure(!(dump && incremental), "Incremental dump not supported");
try {
cctx.kernalContext().security().authorize(ADMIN_SNAPSHOT);
@@ -2167,12 +2213,12 @@ public IgniteFutureImpl createSnapshot(
return new IgniteSnapshotFutureImpl(cctx.kernalContext().closure()
.callAsync(
BALANCE,
- new CreateSnapshotCallable(name, incremental, onlyPrimary),
+ new CreateSnapshotCallable(name, incremental, onlyPrimary, dump),
options(Collections.singletonList(crd)).withFailoverDisabled()
));
}
- if (!CU.isPersistenceEnabled(cctx.gridConfig())) {
+ if (!CU.isPersistenceEnabled(cctx.gridConfig()) && !dump) {
throw new IgniteException("Create snapshot request has been rejected. " +
"Snapshots on an in-memory clusters are not allowed.");
}
@@ -2227,12 +2273,15 @@ public IgniteFutureImpl createSnapshot(
lastSeenSnpFut = snpFut0;
}
- List grps = cctx.cache().persistentGroups().stream()
+ List grps = (dump ? cctx.cache().cacheGroupDescriptors().values() : cctx.cache().persistentGroups()).stream()
.filter(g -> cctx.cache().cacheType(g.cacheOrGroupName()) == CacheType.USER)
.map(CacheGroupDescriptor::cacheOrGroupName)
.collect(Collectors.toList());
- grps.add(METASTORAGE_CACHE_NAME);
+ if (!dump)
+ grps.add(METASTORAGE_CACHE_NAME);
+ else if (grps.isEmpty())
+ throw new IgniteException("Dump operation has been rejected. No cache group defined in cluster");
List srvNodes = cctx.discovery().serverNodes(AffinityTopologyVersion.NONE);
@@ -2259,7 +2308,8 @@ public IgniteFutureImpl createSnapshot(
bltNodeIds,
incremental,
incIdx,
- onlyPrimary
+ onlyPrimary,
+ dump
));
String msg =
@@ -2408,7 +2458,7 @@ public IgniteFutureImpl restoreSnapshot(
if (INC_SNP_NAME_PATTERN.matcher(snpDir.getName()).matches() && snpDir.getAbsolutePath().contains(INC_SNP_DIR))
U.delete(snpDir);
else
- deleteSnapshot(snpDir, pdsSettings.folderName());
+ deleteSnapshot(snpDir, pdsSettings);
if (log.isInfoEnabled()) {
log.info("Previous attempt to create snapshot fail due to the local node crash. All resources " +
@@ -2504,7 +2554,10 @@ public IgniteInternalFuture requestRemoteSnapshotFiles(
* @param grps List of cache groups which will be destroyed.
*/
public void onCacheGroupsStopped(List grps) {
- for (AbstractSnapshotFutureTask> sctx : F.view(locSnpTasks.values(), t -> t instanceof SnapshotFutureTask)) {
+ Collection> tasks =
+ F.view(locSnpTasks.values(), t -> t instanceof SnapshotFutureTask || t instanceof CreateDumpFutureTask);
+
+ for (AbstractSnapshotFutureTask> sctx : tasks) {
Set retain = new HashSet<>(grps);
retain.retainAll(sctx.affectedCacheGroups());
@@ -2636,6 +2689,7 @@ public GridCloseableIterator partitionRowIterator(String snpName,
* @param requestId Snapshot operation request ID.
* @param parts Collection of pairs group and appropriate cache partition to be snapshot.
* @param withMetaStorage {@code true} if all metastorage data must be also included into snapshot.
+ * @param dump {@code true} if cache group dump must be created.
* @param snpSndr Factory which produces snapshot receiver instance.
* @return Snapshot operation task which should be registered on checkpoint to run.
*/
@@ -2645,10 +2699,12 @@ AbstractSnapshotFutureTask> registerSnapshotTask(
UUID requestId,
Map> parts,
boolean withMetaStorage,
+ boolean dump,
SnapshotSender snpSndr
) {
- AbstractSnapshotFutureTask> task = registerTask(snpName, new SnapshotFutureTask(cctx, srcNodeId, requestId,
- snpName, tmpWorkDir, ioFactory, snpSndr, parts, withMetaStorage, locBuff));
+ AbstractSnapshotFutureTask> task = registerTask(snpName, dump
+ ? new CreateDumpFutureTask(cctx, srcNodeId, requestId, snpName, snapshotLocalDir(snpName, null), ioFactory, snpSndr, parts)
+ : new SnapshotFutureTask(cctx, srcNodeId, requestId, snpName, tmpWorkDir, ioFactory, snpSndr, parts, withMetaStorage, locBuff));
if (!withMetaStorage) {
for (Integer grpId : parts.keySet()) {
@@ -2784,6 +2840,22 @@ private void removeLastMetaStorageKey() throws IgniteCheckedException {
}
}
+ /** */
+ private void removeDumpLock(String dumpName) throws IgniteCheckedException {
+ File lock = new File(nodeDumpDirectory(snapshotLocalDir(dumpName, null), cctx), DUMP_LOCK);
+
+ if (!lock.exists())
+ return;
+
+ if (!lock.delete())
+ throw new IgniteCheckedException("Lock file can't be deleted: " + lock);
+ }
+
+ /** */
+ public static File nodeDumpDirectory(File dumpDir, GridCacheSharedContext, ?> cctx) throws IgniteCheckedException {
+ return new File(dumpDir, databaseRelativePath(cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName()));
+ }
+
/**
* Disables creation of incremental snapshots for the given cache group.
*
@@ -3775,8 +3847,6 @@ private Set activeTasks() {
reqMsg0.requestId(),
snpName,
reqMsg0.snapshotPath(),
- tmpWorkDir,
- ioFactory,
rmtSndrFactory.apply(rqId, nodeId),
reqMsg0.parts()));
@@ -4220,7 +4290,7 @@ public LocalSnapshotSender(String snpName, @Nullable String snpPath) {
log.info("The Local snapshot sender closed. All resources released [dbNodeSnpDir=" + dbDir + ']');
}
else {
- deleteSnapshot(snpLocDir, pdsSettings.folderName());
+ deleteSnapshot(snpLocDir, pdsSettings);
if (log.isDebugEnabled())
log.debug("Local snapshot sender closed due to an error occurred: " + th.getMessage());
@@ -4538,25 +4608,44 @@ private static class CreateSnapshotCallable implements IgniteCallable {
/** If {@code true} snapshot only primary copies of partitions. */
private final boolean onlyPrimary;
+ /** If {@code true} create cache dump. */
+ private final boolean dump;
+
/** Auto-injected grid instance. */
@IgniteInstanceResource
private transient IgniteEx ignite;
/**
* @param snpName Snapshot name.
+ * @param incremental If {@code true} then incremental snapshot must be created.
+ * @param onlyPrimary If {@code true} then only copy of primary partitions will be created.
+ * @param dump If {@code true} then cache dump must be created.
*/
- public CreateSnapshotCallable(String snpName, boolean incremental, boolean onlyPrimary) {
+ public CreateSnapshotCallable(
+ String snpName,
+ boolean incremental,
+ boolean onlyPrimary,
+ boolean dump
+ ) {
this.snpName = snpName;
this.incremental = incremental;
this.onlyPrimary = onlyPrimary;
+ this.dump = dump;
}
/** {@inheritDoc} */
- @Override public Void call() throws Exception {
+ @Override public Void call() {
if (incremental)
ignite.snapshot().createIncrementalSnapshot(snpName).get();
- else
- ignite.context().cache().context().snapshotMgr().createSnapshot(snpName, null, false, onlyPrimary).get();
+ else {
+ ignite.context().cache().context().snapshotMgr().createSnapshot(
+ snpName,
+ null,
+ false,
+ onlyPrimary,
+ dump
+ ).get();
+ }
return null;
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java
index 1ffb31fe9a575..6d877b760d3ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotFutureTask.java
@@ -35,7 +35,6 @@
import org.apache.ignite.internal.pagemem.wal.record.IncrementalSnapshotFinishRecord;
import org.apache.ignite.internal.pagemem.wal.record.delta.ClusterSnapshotRecord;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
import org.apache.ignite.internal.util.typedef.internal.CU;
@@ -75,8 +74,6 @@ public IncrementalSnapshotFutureTask(
SnapshotMetadata meta,
@Nullable String snpPath,
int incIdx,
- File tmpWorkDir,
- FileIOFactory ioFactory,
WALPointer lowPtr,
IgniteInternalFuture highPtrFut
) {
@@ -85,8 +82,6 @@ public IncrementalSnapshotFutureTask(
srcNodeId,
reqNodeId,
meta.snapshotName(),
- tmpWorkDir,
- ioFactory,
new SnapshotSender(
cctx.logger(IncrementalSnapshotFutureTask.class),
cctx.kernalContext().pools().getSnapshotExecutorService()
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java
index 2ce074d31e8f9..6cde9cfff443d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IncrementalSnapshotVerificationTask.java
@@ -48,6 +48,7 @@
import org.apache.ignite.internal.processors.cache.GridCacheOperation;
import org.apache.ignite.internal.processors.cache.GridLocalConfigManager;
import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.VerifyPartitionContext;
import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2;
import org.apache.ignite.internal.processors.cache.verify.TransactionsHashRecord;
import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -374,15 +375,10 @@ else if (txRec.state() == TransactionState.ROLLED_BACK) {
e.getKey(),
false,
consId,
- e.getValue().hash,
- e.getValue().verHash,
null,
0,
null,
- 0,
- 0,
- 0,
- 0
+ new VerifyPartitionContext(e.getValue())
)
));
@@ -453,12 +449,12 @@ private Map readTxCachesData() throws IgniteCheckedExc
}
/** Holder for calculated hashes. */
- private static class HashHolder {
+ public static class HashHolder {
/** */
- private int hash;
+ public int hash;
/** */
- private int verHash;
+ public int verHash;
/** */
public void increment(int hash, int verHash) {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
index e32ddc1d00b57..1ba5100f66a47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFinishedFutureTask.java
@@ -25,7 +25,7 @@ public class SnapshotFinishedFutureTask extends AbstractSnapshotFutureTask
* @param e Finished snapshot task future with particular exception.
*/
public SnapshotFinishedFutureTask(IgniteCheckedException e) {
- super(null, null, null, null, null, null, null, null);
+ super(null, null, null, null, null, null);
onDone(e);
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
index 793316949f2d2..ddae6250e039e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java
@@ -24,10 +24,7 @@
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;
@@ -35,7 +32,6 @@
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutionException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicIntegerArray;
import java.util.concurrent.atomic.AtomicLong;
@@ -47,9 +43,7 @@
import java.util.function.BooleanSupplier;
import java.util.stream.Collectors;
import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.binary.BinaryType;
import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteFutureCancelledCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.pagemem.PageIdUtils;
import org.apache.ignite.internal.pagemem.store.PageStore;
@@ -57,8 +51,6 @@
import org.apache.ignite.internal.pagemem.wal.record.delta.ClusterSnapshotRecord;
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.persistence.GridCacheDatabaseSharedManager;
import org.apache.ignite.internal.processors.cache.persistence.checkpoint.CheckpointListener;
import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
@@ -70,20 +62,16 @@
import org.apache.ignite.internal.processors.cache.persistence.wal.WALPointer;
import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
import org.apache.ignite.internal.processors.compress.CompressionProcessor;
-import org.apache.ignite.internal.processors.marshaller.MappedName;
import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageImpl;
import org.apache.ignite.internal.util.GridUnsafe;
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.C3;
-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.jetbrains.annotations.Nullable;
-import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
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.copy;
@@ -99,13 +87,19 @@
* If partitions for particular cache group are not provided that they will be collected and added
* on checkpoint under the write-lock.
*/
-class SnapshotFutureTask extends AbstractSnapshotFutureTask implements CheckpointListener {
+class SnapshotFutureTask extends AbstractCreateSnapshotFutureTask implements CheckpointListener {
/** File page store manager for accessing cache group associated files. */
private final FilePageStoreManager pageStore;
/** Local buffer to perform copy-on-write operations for {@link PageStoreSerialWriter}. */
private final ThreadLocal locBuff;
+ /** Snapshot working directory on file system. */
+ private final File tmpSnpWorkDir;
+
+ /** IO factory which will be used for creating snapshot delta-writers. */
+ private final FileIOFactory ioFactory;
+
/**
* The length of file size per each cache partition file.
* Partition has value greater than zero only for partitions in OWNING state.
@@ -130,9 +124,6 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask> processed = new HashMap<>();
-
/** Checkpoint end future. */
private final CompletableFuture cpEndFut = new CompletableFuture<>();
@@ -142,9 +133,6 @@ class SnapshotFutureTask extends AbstractSnapshotFutureTask closeFut;
-
/** Pointer to {@link ClusterSnapshotRecord}. */
private volatile @Nullable WALPointer snpPtr;
@@ -184,7 +172,7 @@ public SnapshotFutureTask(
boolean withMetaStorage,
ThreadLocal locBuff
) {
- super(cctx, srcNodeId, reqId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
+ super(cctx, srcNodeId, reqId, snpName, snpSndr, parts);
assert snpName != null : "Snapshot name cannot be empty or null.";
assert snpSndr != null : "Snapshot sender which handles execution tasks must be not null.";
@@ -192,6 +180,8 @@ public SnapshotFutureTask(
assert cctx.pageStore() instanceof FilePageStoreManager : "Snapshot task can work only with physical files.";
assert !parts.containsKey(MetaStorage.METASTORAGE_CACHE_ID) : "The withMetaStorage must be used instead.";
+ this.tmpSnpWorkDir = new File(tmpWorkDir, snpName);
+ this.ioFactory = ioFactory;
this.withMetaStorage = withMetaStorage;
this.pageStore = (FilePageStoreManager)cctx.pageStore();
this.locBuff = locBuff;
@@ -204,13 +194,9 @@ public SnapshotFutureTask(
if (th == null)
return;
- if (err.compareAndSet(null, th))
- closeAsync();
+ super.acceptException(th);
startedFut.onDone(th);
-
- if (!(th instanceof IgniteFutureCancelledCheckedException))
- U.error(log, "Snapshot task has accepted exception to stop", th);
}
/** {@inheritDoc} */
@@ -248,13 +234,6 @@ public IgniteInternalFuture> started() {
return startedFut;
}
- /**
- * @return {@code true} if current task requested to be stopped.
- */
- private boolean stopping() {
- return err.get() != null;
- }
-
/**
* Initiates snapshot task.
*
@@ -359,66 +338,7 @@ private boolean stopping() {
ctx.walFlush(true);
}
- for (Map.Entry> e : parts.entrySet()) {
- int grpId = e.getKey();
- Set grpParts = e.getValue();
-
- CacheGroupContext gctx = cctx.cache().cacheGroup(grpId);
-
- Iterator iter;
-
- if (grpParts == null)
- iter = gctx.topology().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, gctx.topology()::localPartition, false);
- }
-
- Set owning = new HashSet<>();
- Set missed = new HashSet<>();
-
- // Iterate over partitions in particular cache group.
- while (iter.hasNext()) {
- GridDhtLocalPartition part = iter.next();
-
- // Partition can be in 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());
- }
-
- boolean affNode = gctx.nodeFilter() == null || gctx.nodeFilter().apply(cctx.localNode());
-
- if (grpParts != null) {
- // Partition has been provided for cache group, but some of them are not in OWNING state.
- // Exit with an error.
- if (!missed.isEmpty()) {
- throw new IgniteCheckedException("Snapshot operation cancelled due to " +
- "not all of requested partitions has OWNING state on local node [grpId=" + grpId +
- ", missed=" + S.toStringSortedDistinct(missed) + ']');
- }
- }
- else {
- // Partitions have not been provided for snapshot task and all partitions have
- // OWNING state, so index partition must be included into snapshot.
- if (!missed.isEmpty()) {
- log.warning("All local cache group partitions in OWNING state have been included into a snapshot. " +
- "Partitions which have different states skipped. Index partitions has also been skipped " +
- "[snpName=" + snpName + ", grpId=" + grpId + ", missed=" + S.toStringSortedDistinct(missed) + ']');
- }
- else if (affNode && missed.isEmpty() && cctx.kernalContext().query().moduleEnabled())
- owning.add(INDEX_PARTITION);
- }
-
- processed.put(grpId, owning);
- }
+ processPartitions();
List> ccfgs = new ArrayList<>();
@@ -465,118 +385,81 @@ else if (affNode && missed.isEmpty() && cctx.kernalContext().query().moduleEnabl
if (!startedFut.onDone())
return;
- // Submit all tasks for partitions and deltas processing.
- List> futs = new ArrayList<>();
-
if (log.isInfoEnabled()) {
log.info("Submit partition processing tasks to the snapshot execution pool " +
"[map=" + groupByGroupId(partFileLengths.keySet()) +
", totalSize=" + U.humanReadableByteCount(partFileLengths.values().stream().mapToLong(v -> v).sum()) + ']');
}
- Collection binTypesCopy = cctx.kernalContext()
- .cacheObjects()
- .metadata(Collections.emptyList())
- .values();
-
- // Process binary meta.
- futs.add(CompletableFuture.runAsync(
- wrapExceptionIfStarted(() -> snpSndr.sendBinaryMeta(binTypesCopy)),
- snpSndr.executor()));
-
- List> mappingsCopy = cctx.kernalContext()
- .marshallerContext()
- .getCachedMappings();
-
- // Process marshaller meta.
- futs.add(CompletableFuture.runAsync(
- wrapExceptionIfStarted(() -> snpSndr.sendMarshallerMeta(mappingsCopy)),
- snpSndr.executor()));
-
- // Send configuration files of all cache groups.
- for (CacheConfigurationSender ccfgSndr : ccfgSndrs)
- futs.add(CompletableFuture.runAsync(wrapExceptionIfStarted(ccfgSndr::sendCacheConfig), snpSndr.executor()));
+ saveSnapshotData();
+ }
- try {
- for (Map.Entry> e : processed.entrySet()) {
- int grpId = e.getKey();
- String cacheDirName = pageStore.cacheDirName(grpId);
+ /** {@inheritDoc} */
+ @Override protected List> saveGroup(int grpId, Set grpParts) throws IgniteCheckedException {
+ String cacheDirName = pageStore.cacheDirName(grpId);
- // Process partitions for a particular cache group.
- for (int partId : e.getValue()) {
- GroupPartitionId pair = new GroupPartitionId(grpId, partId);
+ // Process partitions for a particular cache group.
+ return grpParts.stream().map(partId -> {
+ GroupPartitionId pair = new GroupPartitionId(grpId, partId);
- Long partLen = partFileLengths.get(pair);
+ Long partLen = partFileLengths.get(pair);
- totalSize.addAndGet(partLen);
+ totalSize.addAndGet(partLen);
- CompletableFuture fut0 = CompletableFuture.runAsync(
- wrapExceptionIfStarted(() -> {
- snpSndr.sendPart(
- getPartitionFile(pageStore.workDir(), cacheDirName, partId),
- cacheDirName,
- pair,
- partLen);
+ return runAsync(() -> {
+ snpSndr.sendPart(
+ getPartitionFile(pageStore.workDir(), cacheDirName, partId),
+ cacheDirName,
+ pair,
+ partLen);
- // Stop partition writer.
- partDeltaWriters.get(pair).markPartitionProcessed();
+ // Stop partition writer.
+ partDeltaWriters.get(pair).markPartitionProcessed();
- processedSize.addAndGet(partLen);
- }),
- snpSndr.executor())
- // Wait for the completion of both futures - checkpoint end, copy partition.
- .runAfterBothAsync(cpEndFut,
- wrapExceptionIfStarted(() -> {
- PageStoreSerialWriter writer = partDeltaWriters.get(pair);
+ processedSize.addAndGet(partLen);
- writer.close();
+ // Wait for the completion of both futures - checkpoint end, copy partition.
+ }).runAfterBothAsync(cpEndFut, wrapExceptionIfStarted(() -> {
+ PageStoreSerialWriter writer = partDeltaWriters.get(pair);
- File delta = writer.deltaFile;
+ writer.close();
- try {
- // Atomically creates a new, empty delta file if and only if
- // a file with this name does not yet exist.
- delta.createNewFile();
- }
- catch (IOException ex) {
- throw new IgniteCheckedException(ex);
- }
+ File delta = writer.deltaFile;
- snpSndr.sendDelta(delta, cacheDirName, pair);
+ try {
+ // Atomically creates a new, empty delta file if and only if
+ // a file with this name does not yet exist.
+ delta.createNewFile();
+ }
+ catch (IOException ex) {
+ throw new IgniteCheckedException(ex);
+ }
- processedSize.addAndGet(delta.length());
+ snpSndr.sendDelta(delta, cacheDirName, pair);
- boolean deleted = delta.delete();
+ processedSize.addAndGet(delta.length());
- assert deleted;
+ boolean deleted = delta.delete();
- File deltaIdx = partDeltaIndexFile(delta);
+ assert deleted;
- if (deltaIdx.exists()) {
- deleted = deltaIdx.delete();
+ File deltaIdx = partDeltaIndexFile(delta);
- assert deleted;
- }
- }),
- snpSndr.executor());
+ if (deltaIdx.exists()) {
+ deleted = deltaIdx.delete();
- futs.add(fut0);
+ assert deleted;
}
- }
-
- int futsSize = futs.size();
-
- CompletableFuture.allOf(futs.toArray(new CompletableFuture[futsSize]))
- .whenComplete((res, t) -> {
- assert t == null : "Exception must never be thrown since a wrapper is used " +
- "for each snapshot task: " + t;
+ }), snpSndr.executor());
+ }).collect(Collectors.toList());
+ }
- closeAsync();
- });
- }
- catch (IgniteCheckedException e) {
- acceptException(e);
- }
+ /** {@inheritDoc} */
+ @Override protected List> saveCacheConfigs() {
+ // Send configuration files of all cache groups.
+ return ccfgSndrs.stream()
+ .map(ccfgSndr -> runAsync(ccfgSndr::sendCacheConfig))
+ .collect(Collectors.toList());
}
/**
@@ -585,7 +468,7 @@ else if (affNode && missed.isEmpty() && cctx.kernalContext().query().moduleEnabl
* @param dirName Directory name to init.
* @throws IgniteCheckedException If fails.
*/
- private void addPartitionWriters(int grpId, Set parts, String dirName) throws IgniteCheckedException {
+ void addPartitionWriters(int grpId, Set parts, String dirName) throws IgniteCheckedException {
Integer encGrpId = cctx.cache().isEncrypted(grpId) ? grpId : null;
for (int partId : parts) {
@@ -600,28 +483,8 @@ private void addPartitionWriters(int grpId, Set parts, String dirName)
}
}
- /**
- * @param exec Runnable task to execute.
- * @return Wrapped task.
- */
- private Runnable wrapExceptionIfStarted(IgniteThrowableRunner exec) {
- return () -> {
- if (stopping())
- return;
-
- try {
- exec.run();
- }
- catch (Throwable t) {
- acceptException(t);
- }
- };
- }
-
- /**
- * @return Future which will be completed when operations truly stopped.
- */
- public synchronized CompletableFuture closeAsync() {
+ /** {@inheritDoc} */
+ @Override public synchronized CompletableFuture closeAsync() {
if (closeFut == null) {
Throwable err0 = err.get();
@@ -648,22 +511,6 @@ public long processedSize() {
return processedSize.get();
}
- /** {@inheritDoc} */
- @Override public boolean cancel() {
- super.cancel();
-
- try {
- closeAsync().get();
- }
- catch (InterruptedException | ExecutionException e) {
- U.error(log, "SnapshotFutureTask cancellation failed", e);
-
- return false;
- }
-
- return true;
- }
-
/**
* @param grps List of processing pairs.
*
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTaskResult.java
index 0f2acac059e99..1203825b72363 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTaskResult.java
@@ -27,7 +27,7 @@
/**
* Represents result of {@link SnapshotFutureTask}.
*/
-class SnapshotFutureTaskResult {
+public class SnapshotFutureTaskResult {
/** Partitions for which snapshot was created. */
private final Set parts;
@@ -35,7 +35,7 @@ class SnapshotFutureTaskResult {
private final @Nullable WALPointer snpPtr;
/** */
- SnapshotFutureTaskResult(Set parts, @Nullable WALPointer snpPtr) {
+ public SnapshotFutureTaskResult(Set parts, @Nullable WALPointer snpPtr) {
this.parts = Collections.unmodifiableSet(parts);
this.snpPtr = snpPtr;
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
index 2f2a3abb9dd35..cd04a5f80efe6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java
@@ -101,6 +101,9 @@ public class SnapshotMetadata implements Serializable {
/** If {@code true} snapshot only primary copies of partitions. */
private boolean onlyPrimary;
+ /** If {@code true} cache group dump stored. */
+ private boolean dump;
+
/**
* @param rqId Unique request id.
* @param snpName Snapshot name.
@@ -112,6 +115,7 @@ public class SnapshotMetadata implements Serializable {
* @param snpRecPtr WAL pointer to {@link ClusterSnapshotRecord} if exists.
* @param masterKeyDigest Master key digest for encrypted caches.
* @param onlyPrimary If {@code true} snapshot only primary copies of partitions.
+ * @param dump If {@code true} cache group dump stored.
*/
public SnapshotMetadata(
UUID rqId,
@@ -125,7 +129,8 @@ public SnapshotMetadata(
Set pairs,
@Nullable WALPointer snpRecPtr,
@Nullable byte[] masterKeyDigest,
- boolean onlyPrimary
+ boolean onlyPrimary,
+ boolean dump
) {
this.rqId = rqId;
this.snpName = snpName;
@@ -137,6 +142,7 @@ public SnapshotMetadata(
this.snpRecPtr = snpRecPtr;
this.masterKeyDigest = masterKeyDigest;
this.onlyPrimary = onlyPrimary;
+ this.dump = dump;
if (!F.isEmpty(compGrpIds)) {
hasComprGrps = true;
@@ -228,6 +234,11 @@ public boolean onlyPrimary() {
return onlyPrimary;
}
+ /** @return If {@code true} then metadata describes cache dump. */
+ public boolean dump() {
+ return dump;
+ }
+
/** Save the state of this HashMap partitions and cache groups to a stream. */
private void writeObject(java.io.ObjectOutputStream s)
throws java.io.IOException {
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 05008820bf4f1..90777b1cc560c 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
@@ -91,6 +91,9 @@ public class SnapshotOperationRequest implements Serializable {
/** If {@code true} snapshot only primary copies of partitions. */
private final boolean onlyPrimary;
+ /** If {@code true} then create dump. */
+ private final boolean dump;
+
/**
* @param reqId Request ID.
* @param opNodeId Operational node ID.
@@ -101,6 +104,7 @@ public class SnapshotOperationRequest implements Serializable {
* @param incremental {@code True} if incremental snapshot requested.
* @param incIdx Incremental snapshot index.
* @param onlyPrimary If {@code true} snapshot only primary copies of partitions.
+ * @param dump If {@code true} then create dump.
*/
public SnapshotOperationRequest(
UUID reqId,
@@ -111,7 +115,8 @@ public SnapshotOperationRequest(
Set nodes,
boolean incremental,
int incIdx,
- boolean onlyPrimary
+ boolean onlyPrimary,
+ boolean dump
) {
this.reqId = reqId;
this.opNodeId = opNodeId;
@@ -122,6 +127,7 @@ public SnapshotOperationRequest(
this.incremental = incremental;
this.incIdx = incIdx;
this.onlyPrimary = onlyPrimary;
+ this.dump = dump;
startTime = U.currentTimeMillis();
}
@@ -196,6 +202,11 @@ public boolean onlyPrimary() {
return onlyPrimary;
}
+ /** @return If {@code true} then create dump. */
+ public boolean dump() {
+ return dump;
+ }
+
/** @return Start time. */
public long startTime() {
return startTime;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
index e24a51c6b084a..3ea6d4c126441 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java
@@ -38,7 +38,7 @@
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.dump.DumpEntry;
import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.management.cache.IdleVerifyResultV2;
import org.apache.ignite.internal.management.cache.PartitionKeyV2;
@@ -46,16 +46,20 @@
import org.apache.ignite.internal.managers.encryption.GroupKey;
import org.apache.ignite.internal.managers.encryption.GroupKeyEncrypted;
import org.apache.ignite.internal.pagemem.store.PageStore;
+import org.apache.ignite.internal.processors.cache.CacheObject;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
import org.apache.ignite.internal.processors.cache.StoredCacheData;
import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager;
import org.apache.ignite.internal.processors.cache.persistence.metastorage.MetaStorage;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.Dump;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusIO;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.BPlusMetaIO;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.PageIO;
import org.apache.ignite.internal.processors.cache.persistence.tree.io.PagePartitionMetaIO;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.VerifyPartitionContext;
import org.apache.ignite.internal.processors.cache.verify.PartitionHashRecordV2;
import org.apache.ignite.internal.processors.compress.CompressionProcessor;
import org.apache.ignite.internal.util.GridStringBuilder;
@@ -71,12 +75,16 @@
import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.fromOrdinal;
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.FILE_SUFFIX;
import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheDirectories;
import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cacheGroupName;
import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.cachePartitionFiles;
import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.partId;
import static org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId.getTypeByPartId;
import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.databaseRelativePath;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask.DUMP_FILE_EXT;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.closeAllComponents;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.startAllComponents;
import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.calculatePartitionHash;
import static org.apache.ignite.internal.processors.cache.verify.IdleVerifyUtility.checkPartitionsPageCrcSum;
@@ -135,7 +143,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext, ?> cctx) {
Set parts = meta.partitions().get(grpId) == null ? Collections.emptySet() :
new HashSet<>(meta.partitions().get(grpId));
- for (File part : cachePartitionFiles(dir)) {
+ for (File part : cachePartitionFiles(dir, meta.dump() ? DUMP_FILE_EXT : FILE_SUFFIX)) {
int partId = partId(part.getName());
if (!parts.remove(partId))
@@ -159,6 +167,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext, ?> cctx) {
", meta=" + meta + ']');
}
+ // This will throw if compression disabled. Calculation before other checks.
boolean punchHoleEnabled = isPunchHoleEnabled(opCtx, grpDirs.keySet());
if (!opCtx.check()) {
@@ -167,6 +176,17 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext, ?> cctx) {
return Collections.emptyMap();
}
+ return meta.dump() ? checkDumpFiles(opCtx, partFiles) : checkSnapshotFiles(opCtx, grpDirs, meta, partFiles, punchHoleEnabled);
+ }
+
+ /** */
+ private Map checkSnapshotFiles(
+ SnapshotHandlerContext opCtx,
+ Map grpDirs,
+ SnapshotMetadata meta,
+ Set partFiles,
+ boolean punchHoleEnabled
+ ) throws IgniteCheckedException {
Map res = new ConcurrentHashMap<>();
ThreadLocal buff = ThreadLocal.withInitial(() -> ByteBuffer.allocateDirect(meta.pageSize())
.order(ByteOrder.nativeOrder()));
@@ -180,8 +200,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext, ?> cctx) {
EncryptionCacheKeyProvider snpEncrKeyProvider = new SnapshotEncryptionKeyProvider(cctx.kernalContext(), grpDirs);
- for (GridComponent comp : snpCtx)
- comp.start();
+ startAllComponents(snpCtx);
try {
U.doInParallel(
@@ -294,8 +313,7 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext, ?> cctx) {
throw t;
}
finally {
- for (GridComponent comp : snpCtx)
- comp.stop(true);
+ closeAllComponents(snpCtx);
}
return res;
@@ -335,6 +353,80 @@ private boolean hasExpiringEntries(
return rootIO.getCount(pageAddr) != 0;
}
+ /** */
+ private Map checkDumpFiles(
+ SnapshotHandlerContext opCtx,
+ Set partFiles
+ ) {
+ try {
+ String consistentId = cctx.kernalContext().pdsFolderResolver().resolveFolders().consistentId().toString();
+
+ try (Dump dump = new Dump(opCtx.snapshotDirectory(), consistentId, true, true, log)) {
+ Collection partitionHashRecordV2s = U.doInParallel(
+ cctx.snapshotMgr().snapshotExecutorService(),
+ partFiles,
+ part -> calculateDumpedPartitionHash(dump, cacheGroupName(part.getParentFile()), partId(part.getName()))
+ );
+
+ return partitionHashRecordV2s.stream().collect(Collectors.toMap(PartitionHashRecordV2::partitionKey, r -> r));
+ }
+ catch (Throwable t) {
+ log.error("Error executing handler: ", t);
+
+ throw new IgniteException(t);
+ }
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /** */
+ private PartitionHashRecordV2 calculateDumpedPartitionHash(Dump dump, String grpName, int part) {
+ if (skipHash()) {
+ return new PartitionHashRecordV2(
+ new PartitionKeyV2(CU.cacheId(grpName), part, grpName),
+ false,
+ cctx.localNode().consistentId(),
+ null,
+ 0,
+ PartitionHashRecordV2.PartitionState.OWNING,
+ new VerifyPartitionContext()
+ );
+ }
+
+ try {
+ String node = cctx.kernalContext().pdsFolderResolver().resolveFolders().folderName();
+
+ try (Dump.DumpedPartitionIterator iter = dump.iterator(node, CU.cacheId(grpName), part)) {
+ long size = 0;
+
+ VerifyPartitionContext ctx = new VerifyPartitionContext();
+
+ while (iter.hasNext()) {
+ DumpEntry e = iter.next();
+
+ ctx.update((KeyCacheObject)e.key(), (CacheObject)e.value(), null);
+
+ size++;
+ }
+
+ return new PartitionHashRecordV2(
+ new PartitionKeyV2(CU.cacheId(grpName), part, grpName),
+ false,
+ cctx.localNode().consistentId(),
+ null,
+ size,
+ PartitionHashRecordV2.PartitionState.OWNING,
+ ctx
+ );
+ }
+ }
+ catch (Exception e) {
+ throw new IgniteException(e);
+ }
+ }
+
/** {@inheritDoc} */
@Override public void complete(String name,
Collection>> results) throws IgniteCheckedException {
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
index 90e562f426559..ddb8332e2f486 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotResponseRemoteFutureTask.java
@@ -28,7 +28,6 @@
import java.util.function.Function;
import org.apache.ignite.IgniteException;
import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
-import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
import org.apache.ignite.internal.processors.cache.persistence.partstate.GroupPartitionId;
import org.apache.ignite.internal.util.typedef.F;
import org.jetbrains.annotations.Nullable;
@@ -48,8 +47,6 @@ public class SnapshotResponseRemoteFutureTask extends AbstractSnapshotFutureTask
* @param reqId Snapshot operation request ID.
* @param snpName Unique identifier of snapshot process.
* @param snpPath Snapshot directory path.
- * @param tmpWorkDir Working directory for intermediate snapshot results.
- * @param ioFactory Factory to working with snapshot files.
* @param snpSndr Factory which produces snapshot receiver instance.
* @param parts Partition to be processed.
*/
@@ -59,12 +56,10 @@ public SnapshotResponseRemoteFutureTask(
UUID reqId,
String snpName,
String snpPath,
- File tmpWorkDir,
- FileIOFactory ioFactory,
SnapshotSender snpSndr,
Map> parts
) {
- super(cctx, srcNodeId, reqId, snpName, tmpWorkDir, ioFactory, snpSndr, parts);
+ super(cctx, srcNodeId, reqId, snpName, snpSndr, parts);
this.snpPath = snpPath;
}
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 074a1b7227e16..7fa931d586433 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
@@ -420,7 +420,8 @@ public IgniteFutureImpl start(
new HashSet<>(bltNodes),
false,
incIdx,
- onlyPrimary
+ onlyPrimary,
+ false
);
prepareRestoreProc.start(req.requestId(), req);
@@ -791,8 +792,9 @@ private void enrichContext(
ctx.compress().checkPageCompressionSupported(path.toPath(), meta.pageSize());
}
catch (Exception e) {
- String grpWithCompr = req.groups().stream().filter(s -> meta.isGroupWithCompresion(CU.cacheId(grpName)))
- .collect(Collectors.joining(", "));
+ String grpWithCompr = F.isEmpty(req.groups()) ? ""
+ : req.groups().stream().filter(s -> meta.isGroupWithCompresion(CU.cacheId(grpName)))
+ .collect(Collectors.joining(", "));
String msg = "Requested cache groups [" + grpWithCompr + "] for restore " +
"from snapshot '" + meta.snapshotName() + "' are compressed while " +
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotSender.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotSender.java
index c48f899fc7db9..010ade1952fea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotSender.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotSender.java
@@ -33,7 +33,7 @@
/**
*
*/
-abstract class SnapshotSender {
+public abstract class SnapshotSender {
/** Busy processing lock. */
private final ReadWriteLock lock = new ReentrantReadWriteLock();
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/CreateDumpFutureTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/CreateDumpFutureTask.java
new file mode 100644
index 0000000000000..237665c2ab0c6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/CreateDumpFutureTask.java
@@ -0,0 +1,600 @@
+/*
+ * 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.dump;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
+import java.util.concurrent.locks.LockSupport;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
+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.partstate.GroupPartitionId;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.AbstractCreateSnapshotFutureTask;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotFutureTaskResult;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotSender;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.lang.GridCloseableIterator;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+import static org.apache.ignite.internal.processors.cache.GridLocalConfigManager.cacheDataFilename;
+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.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.DUMP_LOCK;
+
+/**
+ * Task creates cache group dump.
+ * Dump is a consistent snapshot of cache entries.
+ * Directories structure is same as a full snapshot but each partitions saved in "part-0.dump" file.
+ * Files structure is a set of {@link DumpEntry} written one by one.
+ *
+ * @see Dump
+ * @see DumpEntry
+ */
+public class CreateDumpFutureTask extends AbstractCreateSnapshotFutureTask implements DumpEntryChangeListener {
+ /** Dump files name. */
+ public static final String DUMP_FILE_EXT = ".dump";
+
+ /** Root dump directory. */
+ private final File dumpDir;
+
+ /** */
+ private final FileIOFactory ioFactory;
+
+ /**
+ * Dump contextes.
+ * Key is [group_id, partition_id] combined in single long value.
+ *
+ * @see #toLong(int, int)
+ */
+ private final Map dumpCtxs = new ConcurrentHashMap<>();
+
+ /** Local node is primary for set of group partitions. */
+ private final Map> grpPrimaries = new ConcurrentHashMap<>();
+
+ /**
+ * Map shared across all instances of {@link PartitionDumpContext} and {@link DumpEntrySerializer}.
+ * We use per thread buffer because number of threads is fewer then number of partitions.
+ * Regular count of partitions is {@link RendezvousAffinityFunction#DFLT_PARTITION_COUNT}
+ * and thread is {@link IgniteConfiguration#DFLT_PUBLIC_THREAD_CNT} whic is significantly less.
+ */
+ private final ConcurrentMap thLocBufs = new ConcurrentHashMap<>();
+
+ /**
+ * @param cctx Cache context.
+ * @param srcNodeId Node id which cause snapshot task creation.
+ * @param reqId Snapshot operation request ID.
+ * @param dumpName Dump name.
+ * @param ioFactory IO factory.
+ * @param snpSndr Snapshot sender.
+ * @param parts Parts to dump.
+ */
+ public CreateDumpFutureTask(
+ GridCacheSharedContext, ?> cctx,
+ UUID srcNodeId,
+ UUID reqId,
+ String dumpName,
+ File dumpDir,
+ FileIOFactory ioFactory,
+ SnapshotSender snpSndr,
+ Map> parts
+ ) {
+ super(
+ cctx,
+ srcNodeId,
+ reqId,
+ dumpName,
+ snpSndr,
+ parts
+ );
+
+ this.dumpDir = dumpDir;
+ this.ioFactory = ioFactory;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean start() {
+ try {
+ if (log.isInfoEnabled())
+ log.info("Start cache dump [name=" + snpName + ", grps=" + parts.keySet() + ']');
+
+ createDumpLock();
+
+ processPartitions();
+
+ prepare();
+
+ saveSnapshotData();
+ }
+ catch (IgniteCheckedException | IOException e) {
+ acceptException(e);
+ }
+
+ return false; // Don't wait for checkpoint.
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void processPartitions() throws IgniteCheckedException {
+ super.processPartitions();
+
+ processed.values().forEach(parts -> parts.remove(INDEX_PARTITION));
+ }
+
+ /** Prepares all data structures to dump entries. */
+ private void prepare() throws IOException, IgniteCheckedException {
+ for (Map.Entry> e : processed.entrySet()) {
+ int grp = e.getKey();
+
+ File grpDumpDir = groupDirectory(cctx.cache().cacheGroup(grp));
+
+ if (!grpDumpDir.mkdirs())
+ throw new IgniteCheckedException("Dump directory can't be created: " + grpDumpDir);
+
+ CacheGroupContext gctx = cctx.cache().cacheGroup(grp);
+
+ for (GridCacheContext, ?> cctx : gctx.caches())
+ cctx.dumpListener(this);
+
+ grpPrimaries.put(
+ grp,
+ gctx.affinity().primaryPartitions(gctx.shared().kernalContext().localNodeId(), gctx.affinity().lastVersion())
+ );
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override protected List> saveCacheConfigs() {
+ return processed.keySet().stream().map(grp -> runAsync(() -> {
+ CacheGroupContext gctx = cctx.cache().cacheGroup(grp);
+
+ File grpDir = groupDirectory(gctx);
+
+ IgniteUtils.ensureDirectory(grpDir, "dump group directory", null);
+
+ for (GridCacheContext, ?> cacheCtx : gctx.caches()) {
+ DynamicCacheDescriptor desc = cctx.kernalContext().cache().cacheDescriptor(cacheCtx.cacheId());
+
+ StoredCacheData cacheData = new StoredCacheData(new CacheConfiguration(desc.cacheConfiguration()));
+
+ cacheData.queryEntities(desc.schema().entities());
+ cacheData.sql(desc.sql());
+
+ cctx.cache().configManager().writeCacheData(cacheData, new File(grpDir, cacheDataFilename(cacheData.config())));
+ }
+ })).collect(Collectors.toList());
+ }
+
+ /** {@inheritDoc} */
+ @Override protected List> saveGroup(int grp, Set grpParts) {
+ long start = System.currentTimeMillis();
+
+ AtomicLong entriesCnt = new AtomicLong();
+ AtomicLong writtenEntriesCnt = new AtomicLong();
+ AtomicLong changedEntriesCnt = new AtomicLong();
+
+ String name = cctx.cache().cacheGroup(grp).cacheOrGroupName();
+
+ CacheGroupContext gctx = cctx.kernalContext().cache().cacheGroup(grp);
+
+ if (log.isInfoEnabled())
+ log.info("Start group dump [name=" + name + ", id=" + grp + ']');
+
+ List> futs = grpParts.stream().map(part -> runAsync(() -> {
+ long entriesCnt0 = 0;
+ long writtenEntriesCnt0 = 0;
+
+ try (PartitionDumpContext dumpCtx = dumpContext(grp, part)) {
+ try (GridCloseableIterator rows = gctx.offheap().reservedIterator(part, dumpCtx.topVer)) {
+ if (rows == null)
+ throw new IgniteCheckedException("Partition missing [part=" + part + ']');
+
+ while (rows.hasNext()) {
+ CacheDataRow row = rows.next();
+
+ assert row.partition() == part;
+
+ int cache = row.cacheId() == 0 ? grp : row.cacheId();
+
+ if (dumpCtx.writeForIterator(cache, row.expireTime(), row.key(), row.value(), row.version()))
+ writtenEntriesCnt0++;
+
+ entriesCnt0++;
+ }
+ }
+
+ entriesCnt.addAndGet(entriesCnt0);
+ writtenEntriesCnt.addAndGet(writtenEntriesCnt0);
+ changedEntriesCnt.addAndGet(dumpCtx.changedCnt.intValue());
+
+ if (log.isDebugEnabled()) {
+ log.debug("Finish group partition dump [name=" + name +
+ ", id=" + grp +
+ ", part=" + part +
+ ", time=" + (System.currentTimeMillis() - start) +
+ ", iterEntriesCnt=" + entriesCnt +
+ ", writtenIterEntriesCnt=" + entriesCnt +
+ ", changedEntriesCnt=" + changedEntriesCnt + ']');
+
+ }
+ }
+ })).collect(Collectors.toList());
+
+ int futsSize = futs.size();
+
+ CompletableFuture.allOf(futs.toArray(new CompletableFuture[futsSize])).whenComplete((res, t) -> {
+ clearDumpListener(gctx);
+
+ if (log.isInfoEnabled()) {
+ log.info("Finish group dump [name=" + name +
+ ", id=" + grp +
+ ", time=" + (System.currentTimeMillis() - start) +
+ ", iterEntriesCnt=" + entriesCnt.get() +
+ ", writtenIterEntriesCnt=" + writtenEntriesCnt.get() +
+ ", changedEntriesCnt=" + changedEntriesCnt.get() + ']');
+ }
+ });
+
+ return futs;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void beforeChange(GridCacheContext cctx, KeyCacheObject key, CacheObject val, long expireTime, GridCacheVersion ver) {
+ try {
+ int part = key.partition();
+ int grp = cctx.groupId();
+
+ assert part != -1;
+
+ if (!processed.get(grp).contains(part))
+ return;
+
+ dumpContext(grp, part).writeChanged(cctx.cacheId(), expireTime, key, val, ver);
+ }
+ catch (IgniteException e) {
+ acceptException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override protected synchronized CompletableFuture closeAsync() {
+ if (closeFut == null) {
+ dumpCtxs.values().forEach(PartitionDumpContext::close);
+
+ Throwable err0 = err.get();
+
+ Set taken = new HashSet<>();
+
+ for (Map.Entry> e : processed.entrySet()) {
+ int grp = e.getKey();
+
+ clearDumpListener(cctx.cache().cacheGroup(grp));
+
+ for (Integer part : e.getValue())
+ taken.add(new GroupPartitionId(grp, part));
+ }
+
+ closeFut = CompletableFuture.runAsync(
+ () -> {
+ thLocBufs.clear();
+ onDone(new SnapshotFutureTaskResult(taken, null), err0);
+ },
+ cctx.kernalContext().pools().getSystemExecutorService()
+ );
+ }
+
+ return closeFut;
+ }
+
+ /** */
+ private void clearDumpListener(CacheGroupContext gctx) {
+ for (GridCacheContext, ?> cctx : gctx.caches())
+ cctx.dumpListener(null);
+ }
+
+ /** */
+ private void createDumpLock() throws IgniteCheckedException, IOException {
+ File nodeDumpDir = IgniteSnapshotManager.nodeDumpDirectory(dumpDir, cctx);
+
+ if (!nodeDumpDir.mkdirs())
+ throw new IgniteCheckedException("Can't create node dump directory: " + nodeDumpDir.getAbsolutePath());
+
+ File lock = new File(nodeDumpDir, DUMP_LOCK);
+
+ if (!lock.createNewFile())
+ throw new IgniteCheckedException("Lock file can't be created or already exists: " + lock.getAbsolutePath());
+ }
+
+ /** */
+ private PartitionDumpContext dumpContext(int grp, int part) {
+ return dumpCtxs.computeIfAbsent(
+ toLong(grp, part),
+ key -> new PartitionDumpContext(cctx.kernalContext().cache().cacheGroup(grp), part, thLocBufs)
+ );
+ }
+
+ /**
+ * Context of dump single partition.
+ */
+ private class PartitionDumpContext implements Closeable {
+ /** Group id. */
+ final int grp;
+
+ /** Partition id. */
+ final int part;
+
+ /**
+ * Key is cache id, values is set of keys dumped via
+ * {@link #writeChanged(int, long, KeyCacheObject, CacheObject, GridCacheVersion)}.
+ */
+ final Map> changed;
+
+ /** Count of entries changed during dump creation. */
+ LongAdder changedCnt = new LongAdder();
+
+ /** Partition dump file. Lazily initialized to prevent creation files for empty partitions. */
+ final FileIO file;
+
+ /**
+ * Regular updates with {@link IgniteCache#put(Object, Object)} and similar calls
+ * will use version generated with {@link GridCacheVersionManager#next(GridCacheVersion)}.
+ * Version is monotonically increase.
+ * Version generated on primary node and propagated to backups.
+ * So on primary we can distinguish updates that happens before and after dump start comparing versions
+ * with the version we read with {@link GridCacheVersionManager#last()}.
+ */
+ @Nullable final GridCacheVersion startVer;
+
+ /**
+ * Unlike regular update, {@link IgniteDataStreamer} updates receive the same version for all entries.
+ * See {@code IsolatedUpdater.receive}.
+ * Note, using {@link IgniteDataStreamer} during cache dump creation can lead to dump inconsistency.
+ *
+ * @see GridCacheVersionManager#isolatedStreamerVersion()
+ */
+ final GridCacheVersion isolatedStreamerVer;
+
+ /** Topology Version. */
+ private final AffinityTopologyVersion topVer;
+
+ /** Partition serializer. */
+ private final DumpEntrySerializer serializer;
+
+ /** If {@code true} context is closed. */
+ volatile boolean closed;
+
+ /**
+ * Count of writers. When count becomes {@code 0} context must be closed.
+ * By deafult, one writer exists - partition iterator.
+ * Each call of {@link #writeChanged(int, long, KeyCacheObject, CacheObject, GridCacheVersion)} increment writers count.
+ * When count of writers becomes zero we good to relase all resources associated with partition dump.
+ */
+ private final AtomicInteger writers = new AtomicInteger(1);
+
+ /**
+ * @param gctx Group context.
+ * @param part Partition id.
+ * @param thLocBufs Thread local buffers.
+ */
+ public PartitionDumpContext(CacheGroupContext gctx, int part, ConcurrentMap thLocBufs) {
+ assert gctx != null;
+
+ try {
+ this.part = part;
+ grp = gctx.groupId();
+ topVer = gctx.topology().lastTopologyChangeVersion();
+
+ startVer = grpPrimaries.get(gctx.groupId()).contains(part) ? gctx.shared().versions().last() : null;
+ isolatedStreamerVer = cctx.versions().isolatedStreamerVersion();
+
+ serializer = new DumpEntrySerializer(thLocBufs);
+ changed = new HashMap<>();
+
+ for (int cache : gctx.cacheIds())
+ changed.put(cache, new GridConcurrentHashSet<>());
+
+ File dumpFile = new File(groupDirectory(gctx), PART_FILE_PREFIX + part + DUMP_FILE_EXT);
+
+ if (!dumpFile.createNewFile())
+ throw new IgniteException("Dump file can't be created: " + dumpFile);
+
+ file = ioFactory.create(dumpFile);
+ }
+ catch (IOException | IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /**
+ * Writes entry that changed while dump creation in progress.
+ * @param cache Cache id.
+ * @param expireTime Expire time.
+ * @param key Key.
+ * @param val Value before change.
+ * @param ver Version before change.
+ */
+ public void writeChanged(int cache, long expireTime, KeyCacheObject key, CacheObject val, GridCacheVersion ver) {
+ String reasonToSkip = null;
+
+ if (closed) // Quick exit. Partition already saved in dump.
+ reasonToSkip = "partition already saved";
+ else {
+ writers.getAndIncrement();
+
+ try {
+ if (closed) // Partition already saved in dump.
+ reasonToSkip = "partition already saved";
+ else if (isAfterStart(ver))
+ reasonToSkip = "greater version";
+ else if (!changed.get(cache).add(key)) // Entry changed several time during dump.
+ reasonToSkip = "changed several times";
+ else if (val == null)
+ reasonToSkip = "newly created or already removed"; // Previous value is null. Entry created after dump start, skip.
+ else {
+ write(cache, expireTime, key, val);
+
+ changedCnt.increment();
+ }
+ }
+ finally {
+ writers.decrementAndGet();
+ }
+ }
+
+ if (log.isTraceEnabled()) {
+ log.trace("Listener [grp=" + grp +
+ ", cache=" + cache +
+ ", part=" + part +
+ ", key=" + key +
+ ", written=" + (reasonToSkip == null ? "true" : reasonToSkip) + ']');
+ }
+ }
+
+ /**
+ * Writes entry fetched from partition iterator.
+ *
+ * @param cache Cache id.
+ * @param expireTime Expire time.
+ * @param key Key.
+ * @param val Value.
+ * @param ver Version.
+ * @return {@code True} if entry was written in dump.
+ * {@code false} if it was already written by {@link #writeChanged(int, long, KeyCacheObject, CacheObject, GridCacheVersion)}.
+ */
+ public boolean writeForIterator(
+ int cache,
+ long expireTime,
+ KeyCacheObject key,
+ CacheObject val,
+ GridCacheVersion ver
+ ) {
+ boolean written = true;
+
+ if (isAfterStart(ver))
+ written = false;
+ else if (changed.get(cache).contains(key))
+ written = false;
+ else
+ write(cache, expireTime, key, val);
+
+ if (log.isTraceEnabled()) {
+ log.trace("Iterator [" +
+ "grp=" + grp +
+ ", cache=" + cache +
+ ", part=" + part +
+ ", key=" + key +
+ ", written=" + written + ']');
+ }
+
+ return written;
+ }
+
+ /** */
+ private void write(int cache, long expireTime, KeyCacheObject key, CacheObject val) {
+ synchronized (serializer) { // Prevent concurrent access to the dump file.
+ try {
+ ByteBuffer buf = serializer.writeToBuffer(cache, expireTime, key, val, cctx.cacheObjectContext(cache));
+
+ if (file.writeFully(buf) != buf.limit())
+ throw new IgniteException("Can't write row");
+ }
+ catch (IOException | IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }
+ }
+
+ /**
+ * Note, usage of {@link IgniteDataStreamer} may lead to dump inconsistency.
+ * Because, streamer use the same {@link GridCacheVersion} for all entries.
+ * So, we can't efficiently filter out new entries and write them all to dump.
+ *
+ * @param ver Entry version.
+ * @return {@code True} if {@code ver} appeared after dump started.
+ */
+ private boolean isAfterStart(GridCacheVersion ver) {
+ return (startVer != null && ver.isGreater(startVer)) && !isolatedStreamerVer.equals(ver);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ synchronized (this) { // Prevent concurrent close invocation.
+ if (closed)
+ return;
+
+ closed = true;
+ }
+
+ writers.decrementAndGet();
+
+ while (writers.get() > 0) // Waiting for all on the fly listeners to complete.
+ LockSupport.parkNanos(1_000_000);
+
+ U.closeQuiet(file);
+ }
+ }
+
+ /** */
+ public static long toLong(int high, int low) {
+ return (((long)high) << Integer.SIZE) | (low & 0xffffffffL);
+ }
+
+ /** */
+ private File groupDirectory(CacheGroupContext grpCtx) throws IgniteCheckedException {
+ return new File(
+ IgniteSnapshotManager.nodeDumpDirectory(dumpDir, cctx),
+ (grpCtx.caches().size() > 1 ? CACHE_GRP_DIR_PREFIX : CACHE_DIR_PREFIX) + grpCtx.cacheOrGroupName()
+ );
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/Dump.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/Dump.java
new file mode 100644
index 0000000000000..1874e0727d74b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/Dump.java
@@ -0,0 +1,361 @@
+/*
+ * 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.dump;
+
+import java.io.BufferedInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+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.BinaryObject;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+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.cache.persistence.snapshot.SnapshotMetadata;
+import org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext;
+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.U;
+import org.apache.ignite.marshaller.MarshallerUtils;
+import org.apache.ignite.marshaller.jdk.JdkMarshaller;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_BINARY_METADATA_PATH;
+import static org.apache.ignite.configuration.DataStorageConfiguration.DFLT_MARSHALLER_PATH;
+import static org.apache.ignite.internal.processors.cache.GridLocalConfigManager.readCacheData;
+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.file.FilePageStoreManager.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNAPSHOT_METAFILE_EXT;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask.DUMP_FILE_EXT;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.closeAllComponents;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.startAllComponents;
+
+/**
+ * This class provides ability to work with saved cache dump.
+ */
+public class Dump implements AutoCloseable {
+ /** Snapshot meta. */
+ private final List metadata;
+
+ /** Dump directory. */
+ private final File dumpDir;
+
+ /** Specific consistent id. */
+ private final @Nullable String consistentId;
+
+ /** Kernal context for each node in dump. */
+ private final GridKernalContext cctx;
+
+ /** If {@code true} then return data in form of {@link BinaryObject}. */
+ private final boolean keepBinary;
+
+ /** If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}. */
+ private final boolean raw;
+
+ /**
+ * Map shared across all instances of {@link DumpEntrySerializer}.
+ * We use per thread buffer because number of threads is fewer then number of partitions.
+ * Regular count of partitions is {@link RendezvousAffinityFunction#DFLT_PARTITION_COUNT}
+ * and thread is {@link IgniteConfiguration#DFLT_PUBLIC_THREAD_CNT} whic is significantly less.
+ */
+ private final ConcurrentMap thLocBufs = new ConcurrentHashMap<>();
+
+ /**
+ * @param dumpDir Dump directory.
+ * @param keepBinary If {@code true} then keep read entries in binary form.
+ * @param raw If {@code true} then keep read entries in form of {@link KeyCacheObject} and {@link CacheObject}.
+ * @param log Logger.
+ */
+ public Dump(File dumpDir, boolean keepBinary, boolean raw, IgniteLogger log) {
+ this(dumpDir, null, keepBinary, raw, log);
+ }
+
+ /**
+ * @param dumpDir Dump directory.
+ * @param consistentId If specified, read dump data only for specific node.
+ * @param keepBinary If {@code true} then keep read entries in binary form.
+ * @param raw If {@code true} then keep read entries in form of {@link KeyCacheObject} and {@link CacheObject}.
+ * @param log Logger.
+ */
+ public Dump(File dumpDir, @Nullable String consistentId, boolean keepBinary, boolean raw, IgniteLogger log) {
+ A.ensure(dumpDir != null, "dump directory is null");
+ A.ensure(dumpDir.exists(), "dump directory not exists");
+
+ this.dumpDir = dumpDir;
+ this.consistentId = consistentId == null ? null : U.maskForFileName(consistentId);
+ this.metadata = metadata(dumpDir, this.consistentId);
+ this.keepBinary = keepBinary;
+ this.cctx = standaloneKernalContext(dumpDir, log);
+ this.raw = raw;
+ }
+
+ /**
+ * @param dumpDir Dump directory.
+ * @param log Logger.
+ * @return Standalone kernal context.
+ */
+ private GridKernalContext standaloneKernalContext(File dumpDir, IgniteLogger log) {
+ File binaryMeta = CacheObjectBinaryProcessorImpl.binaryWorkDir(dumpDir.getAbsolutePath(), F.first(metadata).folderName());
+ File marshaller = new File(dumpDir, DFLT_MARSHALLER_PATH);
+
+ A.ensure(binaryMeta.exists(), "binary metadata directory not exists");
+ A.ensure(marshaller.exists(), "marshaller directory not exists");
+
+ try {
+ GridKernalContext kctx = new StandaloneGridKernalContext(log, binaryMeta, marshaller);
+
+ startAllComponents(kctx);
+
+ return kctx;
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /** @return Binary types iterator. */
+ public Iterator types() {
+ return cctx.cacheObjects().metadata().iterator();
+ }
+
+ /** @return List of node directories. */
+ public List nodesDirectories() {
+ File[] dirs = new File(dumpDir, DFLT_STORE_DIR).listFiles(f -> f.isDirectory()
+ && !(f.getAbsolutePath().endsWith(DFLT_BINARY_METADATA_PATH) || f.getAbsolutePath().endsWith(DFLT_MARSHALLER_PATH))
+ && (consistentId == null || U.maskForFileName(f.getName()).contains(consistentId)));
+
+ if (dirs == null)
+ return Collections.emptyList();
+
+ return Arrays.stream(dirs).map(File::getName).collect(Collectors.toList());
+ }
+
+ /** @return List of snapshot metadata saved in {@link #dumpDir}. */
+ public List metadata() throws IOException, IgniteCheckedException {
+ return metadata;
+ }
+
+ /** @return List of snapshot metadata saved in {@link #dumpDir}. */
+ private static List metadata(File dumpDir, @Nullable String consistentId) {
+ JdkMarshaller marsh = MarshallerUtils.jdkMarshaller("fake-node");
+
+ ClassLoader clsLdr = U.resolveClassLoader(new IgniteConfiguration());
+
+ File[] files = dumpDir.listFiles(f ->
+ f.getName().endsWith(SNAPSHOT_METAFILE_EXT) && (consistentId == null || f.getName().startsWith(consistentId))
+ );
+
+ if (files == null)
+ return Collections.emptyList();
+
+ return Arrays.stream(files).map(meta -> {
+ try (InputStream in = new BufferedInputStream(Files.newInputStream(meta.toPath()))) {
+ return marsh.unmarshal(in, clsLdr);
+ }
+ catch (IOException | IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }).filter(SnapshotMetadata::dump).collect(Collectors.toList());
+ }
+
+ /**
+ * @param node Node directory name.
+ * @param group Group id.
+ * @return List of cache configs saved in dump for group.
+ */
+ public List configs(String node, int group) {
+ JdkMarshaller marsh = MarshallerUtils.jdkMarshaller(cctx.igniteInstanceName());
+
+ return Arrays.stream(FilePageStoreManager.cacheDataFiles(dumpGroupDirectory(node, group))).map(f -> {
+ try {
+ return readCacheData(f, marsh, cctx.config());
+ }
+ catch (IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }).collect(Collectors.toList());
+ }
+
+ /**
+ * @param node Node directory name.
+ * @param group Group id.
+ * @return Dump iterator.
+ */
+ public List partitions(String node, int group) {
+ File[] parts = dumpGroupDirectory(node, group)
+ .listFiles(f -> f.getName().startsWith(PART_FILE_PREFIX) && f.getName().endsWith(DUMP_FILE_EXT));
+
+ if (parts == null)
+ return Collections.emptyList();
+
+ return Arrays.stream(parts)
+ .map(partFile -> Integer.parseInt(partFile.getName().replace(PART_FILE_PREFIX, "").replace(DUMP_FILE_EXT, "")))
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * @param node Node directory name.
+ * @param group Group id.
+ * @return Dump iterator.
+ */
+ public DumpedPartitionIterator iterator(String node, int group, int part) {
+ FileIOFactory ioFactory = new RandomAccessFileIOFactory();
+
+ FileIO dumpFile;
+
+ try {
+ dumpFile = ioFactory.create(new File(dumpGroupDirectory(node, group), PART_FILE_PREFIX + part + DUMP_FILE_EXT));
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ DumpEntrySerializer serializer = new DumpEntrySerializer(thLocBufs);
+
+ serializer.kernalContext(cctx);
+ serializer.keepBinary(keepBinary);
+ serializer.raw(raw);
+
+ return new DumpedPartitionIterator() {
+ DumpEntry next;
+
+ Set partKeys = new HashSet<>();
+
+ /** {@inheritDoc} */
+ @Override public boolean hasNext() {
+ advance();
+
+ return next != null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public DumpEntry next() {
+ advance();
+
+ if (next == null)
+ throw new NoSuchElementException();
+
+ DumpEntry next0 = next;
+
+ next = null;
+
+ return next0;
+ }
+
+ /** */
+ private void advance() {
+ if (next != null)
+ return;
+
+ try {
+ next = serializer.read(dumpFile, group, part);
+
+ /*
+ * During dumping entry can be dumped twice: by partition iterator and change listener.
+ * Excluding duplicates keys from iteration.
+ */
+ while (next != null && !partKeys.add(next.key()))
+ next = serializer.read(dumpFile, group, part);
+
+ if (next == null)
+ partKeys = null; // Let GC do the rest.
+ }
+ catch (IOException | IgniteCheckedException e) {
+ throw new IgniteException(e);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ U.closeQuiet(dumpFile);
+
+ partKeys = null;
+ }
+ };
+ }
+
+ /** @return Root dump directory. */
+ public File dumpDirectory() {
+ return dumpDir;
+ }
+
+ /** */
+ private File dumpGroupDirectory(String node, int groupId) {
+ File nodeDir = Paths.get(dumpDir.getAbsolutePath(), DFLT_STORE_DIR, node).toFile();
+
+ assert nodeDir.exists() && nodeDir.isDirectory();
+
+ File[] grpDirs = nodeDir.listFiles(f -> {
+ if (!f.isDirectory()
+ || (!f.getName().startsWith(CACHE_DIR_PREFIX)
+ && !f.getName().startsWith(CACHE_GRP_DIR_PREFIX)))
+ return false;
+
+ String grpName = f.getName().startsWith(CACHE_DIR_PREFIX)
+ ? f.getName().replaceFirst(CACHE_DIR_PREFIX, "")
+ : f.getName().replaceFirst(CACHE_GRP_DIR_PREFIX, "");
+
+ return groupId == CU.cacheId(grpName);
+ });
+
+ if (grpDirs.length != 1)
+ throw new IgniteException("Wrong number of group directories: " + grpDirs.length);
+
+ return grpDirs[0];
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() throws Exception {
+ closeAllComponents(cctx);
+ }
+
+ /**
+ * Closeable dump iterator.
+ */
+ public interface DumpedPartitionIterator extends Iterator, AutoCloseable {
+ // No-op.
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntryChangeListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntryChangeListener.java
new file mode 100644
index 0000000000000..40669266a8c41
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntryChangeListener.java
@@ -0,0 +1,38 @@
+/*
+ * 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.dump;
+
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Copy on write listener that handles changes of entries during dump.
+ */
+public interface DumpEntryChangeListener {
+ /**
+ * @param cctx Cache context.
+ * @param key Key.
+ * @param val Value BEFORE change.
+ * @param expireTime Expire time BEFORE change.
+ * @param ver Version of entry BEFORE change.
+ */
+ public void beforeChange(GridCacheContext cctx, KeyCacheObject key, @Nullable CacheObject val, long expireTime, GridCacheVersion ver);
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntrySerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntrySerializer.java
new file mode 100644
index 0000000000000..27146ec3644ef
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/DumpEntrySerializer.java
@@ -0,0 +1,260 @@
+/*
+ * 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.dump;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.pagemem.wal.record.UnwrapDataEntry;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.wal.crc.FastCrc;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+
+/**
+ * Serialization logic for dump.
+ */
+public class DumpEntrySerializer {
+ /** sizeOf(CRC) + sizeOf(Data size) */
+ public static final int HEADER_SZ = Integer.BYTES + Integer.BYTES;
+
+ /** Default buffer allocator. */
+ private static final Function DFLT_BUF_ALLOC = k -> ByteBuffer.allocate(100);
+
+ /** */
+ private final ConcurrentMap thLocBufs;
+
+ /** */
+ private final FastCrc crc = new FastCrc();
+
+ /** Cache object processor. */
+ private IgniteCacheObjectProcessor co;
+
+ /** Fake context. */
+ private CacheObjectContext fakeCacheObjCtx;
+
+ /** If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}. */
+ private boolean raw;
+
+ /** If {@code true} then return data in form of {@link BinaryObject}. */
+ private boolean keepBinary;
+
+ /**
+ * @param thLocBufs Thread local buffers.
+ */
+ public DumpEntrySerializer(ConcurrentMap thLocBufs) {
+ this.thLocBufs = thLocBufs;
+ }
+
+ /** */
+ public void kernalContext(GridKernalContext cctx) {
+ co = cctx.cacheObjects();
+ fakeCacheObjCtx = new CacheObjectContext(cctx, null, null, false, false, false, false, false);
+ }
+
+ /** @param keepBinary If {@code true} then return data in form of {@link BinaryObject}. */
+ public void keepBinary(boolean keepBinary) {
+ this.keepBinary = keepBinary;
+ }
+
+ /** @param raw If {@code true} then don't deserialize {@link KeyCacheObject} and {@link CacheObject}. */
+ public void raw(boolean raw) {
+ this.raw = raw;
+ }
+
+ /**
+ * Dump entry structure:
+ *
+ * +---------+-----------+----------+-----------------+-----+-------+
+ * | 4 bytes | 4 bytes | 4 bytes | 8 bytes | | |
+ * +---------+-----------+----------+-----------------+-----+-------+
+ * | CRC | Data size | cache ID | expiration time | key | value |
+ * +---------+-----------+----------+-----------------+-----+-------+
+ *
+ *
+ * @param cache Cache id.
+ * @param expireTime Expire time.
+ * @param key Key.
+ * @param val Value.
+ * @param coCtx Cache object context.
+ * @return Buffer with serialized entry.
+ * @throws IgniteCheckedException If failed
+ */
+ public ByteBuffer writeToBuffer(
+ int cache,
+ long expireTime,
+ KeyCacheObject key,
+ CacheObject val,
+ CacheObjectContext coCtx
+ ) throws IgniteCheckedException {
+ int keySz = key.valueBytesLength(coCtx);
+ int valSz = val.valueBytesLength(coCtx);
+ int dataSz = /*cache ID*/Integer.BYTES + /*expire time*/Long.BYTES + /*key*/keySz + /*value*/valSz;
+
+ int fullSz = dataSz + /*extra bytes for row size*/Integer.BYTES + /*CRC*/Integer.BYTES;
+
+ ByteBuffer buf = threadLocalBuffer();
+
+ if (buf.capacity() < fullSz)
+ buf = enlargeThreadLocalBuffer(fullSz);
+ else
+ buf.rewind().limit(fullSz);
+
+ buf.position(Integer.BYTES); // CRC value.
+ buf.putInt(dataSz);
+ buf.putInt(cache);
+ buf.putLong(expireTime);
+
+ if (!key.putValue(buf))
+ throw new IgniteCheckedException("Can't write key");
+
+ if (!val.putValue(buf))
+ throw new IgniteCheckedException("Can't write value");
+
+ assert buf.position() == fullSz;
+
+ buf.position(Integer.BYTES);
+
+ crc.reset();
+ crc.update(buf, fullSz - Integer.BYTES);
+
+ buf.position(0);
+ buf.putInt(crc.getValue());
+
+ buf.position(0);
+
+ return buf;
+ }
+
+ /**
+ * @param dumpFile File to read data from.
+ * @param grp Cache group.
+ * @return dump entry.
+ */
+ public DumpEntry read(FileIO dumpFile, int grp, int part) throws IOException, IgniteCheckedException {
+ assert co != null : "Set kernalContext first";
+
+ ByteBuffer buf = threadLocalBuffer();
+
+ buf.position(0);
+ buf.limit(HEADER_SZ);
+
+ int read = dumpFile.readFully(buf);
+
+ if (read < HEADER_SZ)
+ return null;
+
+ buf.position(0);
+
+ int crc = buf.getInt();
+ int dataSz = buf.getInt();
+
+ if (buf.capacity() < dataSz + HEADER_SZ) {
+ buf = enlargeThreadLocalBuffer(dataSz + HEADER_SZ);
+
+ buf.position(HEADER_SZ - Integer.BYTES);
+ buf.putInt(dataSz); // Required for CRC check.
+ }
+ else
+ buf.limit(dataSz + HEADER_SZ);
+
+ read = dumpFile.readFully(buf);
+
+ if (read != dataSz)
+ throw new IgniteException("Expected to read " + dataSz + " bytes but read only " + read);
+
+ buf.position(HEADER_SZ - Integer.BYTES);
+
+ checkCRC(crc, dataSz, buf);
+
+ buf.position(HEADER_SZ);
+
+ int cache = buf.getInt();
+ long expireTime = buf.getLong();
+
+ int keySz = buf.getInt();
+
+ byte keyType = buf.get();
+
+ byte[] keyBytes = new byte[keySz];
+
+ buf.get(keyBytes, 0, keyBytes.length);
+
+ KeyCacheObject key = co.toKeyCacheObject(fakeCacheObjCtx, keyType, keyBytes);
+
+ if (key.partition() == -1)
+ key.partition(part);
+
+ int valSz = buf.getInt();
+ byte valType = buf.get();
+ byte[] valBytes = new byte[valSz];
+
+ buf.get(valBytes, 0, valBytes.length);
+
+ CacheObject val = co.toCacheObject(fakeCacheObjCtx, valType, valBytes);
+
+ return new DumpEntry() {
+ @Override public int cacheId() {
+ return cache;
+ }
+
+ @Override public long expireTime() {
+ return expireTime;
+ }
+
+ @Override public Object key() {
+ return raw ? key : UnwrapDataEntry.unwrapKey(key, keepBinary, fakeCacheObjCtx);
+ }
+
+ @Override public Object value() {
+ return raw ? val : UnwrapDataEntry.unwrapValue(val, keepBinary, fakeCacheObjCtx);
+ }
+ };
+ }
+
+ /** @return Thread local buffer. */
+ private ByteBuffer threadLocalBuffer() {
+ return thLocBufs.computeIfAbsent(Thread.currentThread().getId(), DFLT_BUF_ALLOC);
+ }
+
+ /** @return Thread local buffer. */
+ private ByteBuffer enlargeThreadLocalBuffer(int sz) {
+ ByteBuffer buf = ByteBuffer.allocate(sz);
+
+ thLocBufs.put(Thread.currentThread().getId(), buf);
+
+ return buf;
+ }
+
+ /** */
+ private void checkCRC(int expCrc, int dataSz, ByteBuffer buf) {
+ crc.reset();
+ crc.update(buf, dataSz + Integer.BYTES /*dataSz field included in CRC calculation*/);
+
+ if (expCrc != crc.getValue())
+ throw new IgniteException("Data corrupted [expCrc=" + expCrc + ", crc=" + crc + ']');
+ }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
index 05c40db53bd8c..d01a2f8413904 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/IgniteWalIteratorFactory.java
@@ -34,7 +34,6 @@
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.configuration.DataStorageConfiguration;
-import org.apache.ignite.internal.GridComponent;
import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.pagemem.wal.WALIterator;
import org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType;
@@ -57,6 +56,8 @@
import static java.nio.file.Files.walkFileTree;
import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_NAME_PATTERN;
import static org.apache.ignite.internal.processors.cache.persistence.wal.FileWriteAheadLogManager.WAL_SEGMENT_FILE_COMPACTED_PATTERN;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.closeAllComponents;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.startAllComponents;
import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.HEADER_RECORD_SIZE;
import static org.apache.ignite.internal.processors.cache.persistence.wal.serializer.RecordV1Serializer.readPosition;
@@ -176,8 +177,7 @@ public WALIterator iterator(
if (iteratorParametersBuilder.sharedCtx == null) {
GridCacheSharedContext, ?> sctx = prepareSharedCtx(iteratorParametersBuilder);
- for (GridComponent comp : sctx.kernalContext())
- comp.start();
+ startAllComponents(sctx.kernalContext());
return new StandaloneWalRecordsIterator(
iteratorParametersBuilder.log == null ? log : iteratorParametersBuilder.log,
@@ -194,8 +194,7 @@ public WALIterator iterator(
@Override protected void onClose() throws IgniteCheckedException {
super.onClose();
- for (GridComponent comp : sctx.kernalContext())
- comp.stop(true);
+ closeAllComponents(sctx.kernalContext());
}
};
}
@@ -690,9 +689,9 @@ private File[] merge(File[] f1, File[] f2) {
/**
*
*/
- private static class ConsoleLogger implements IgniteLogger {
+ public static class ConsoleLogger implements IgniteLogger {
/** */
- private static final ConsoleLogger INSTANCE = new ConsoleLogger();
+ public static final ConsoleLogger INSTANCE = new ConsoleLogger();
/** */
private static final PrintStream OUT = System.out;
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 8c1f52a4c15a1..1103f198dde90 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -694,4 +694,22 @@ private void setField(IgniteEx kernal, String name, Object val) throws NoSuchFie
@Override public Executor getAsyncContinuationExecutor() {
return null;
}
+
+ /**
+ * @param kctx Kernal context.
+ * @throws IgniteCheckedException In case of any error.
+ */
+ public static void startAllComponents(GridKernalContext kctx) throws IgniteCheckedException {
+ for (GridComponent comp : kctx)
+ comp.start();
+ }
+
+ /**
+ * @param kctx Kernal context.
+ * @throws IgniteCheckedException In case of any error.
+ */
+ public static void closeAllComponents(GridKernalContext kctx) throws IgniteCheckedException {
+ for (GridComponent comp : kctx)
+ comp.stop(true);
+ }
}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java
index 79c1806d08509..290dc0352e64a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java
@@ -36,13 +36,17 @@
import org.apache.ignite.internal.pagemem.PageIdAllocator;
import org.apache.ignite.internal.pagemem.PageIdUtils;
import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.CacheObject;
import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
import org.apache.ignite.internal.processors.cache.PartitionUpdateCounter;
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.CacheDataRow;
import org.apache.ignite.internal.processors.cache.persistence.file.FilePageStore;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.IncrementalSnapshotVerificationTask.HashHolder;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
import org.apache.ignite.internal.util.lang.GridIterator;
import org.apache.ignite.internal.util.lang.IgniteThrowableSupplier;
import org.apache.ignite.internal.util.typedef.F;
@@ -278,28 +282,19 @@ public static List compareUpdateCounters(
return new PartitionHashRecordV2(partKey,
isPrimary,
consId,
- 0,
- 0,
updCntr,
state == GridDhtPartitionState.MOVING ?
PartitionHashRecordV2.MOVING_PARTITION_SIZE : 0,
state == GridDhtPartitionState.MOVING ?
PartitionHashRecordV2.PartitionState.MOVING : PartitionHashRecordV2.PartitionState.LOST,
- 0,
- 0,
- 0,
- 0);
+ new VerifyPartitionContext()
+ );
}
if (state != GridDhtPartitionState.OWNING)
return null;
- int partHash = 0;
- int partVerHash = 0;
- int cf = 0;
- int noCf = 0;
- int binary = 0;
- int regular = 0;
+ VerifyPartitionContext ctx = new VerifyPartitionContext();
while (it.hasNextX()) {
CacheDataRow row = it.nextX();
@@ -307,37 +302,17 @@ public static List compareUpdateCounters(
if (row.expireTime() > 0)
continue;
- partHash += row.key().hashCode();
- partVerHash += row.version().hashCode(); // Detects ABA problem.
-
- // Object context is not required since the valueBytes have been read directly from page.
- partHash += Arrays.hashCode(row.value().valueBytes(null));
-
- if (row.key().cacheObjectType() == TYPE_BINARY) {
- binary++;
-
- if (((BinaryObjectEx)row.key()).isFlagSet(FLAG_COMPACT_FOOTER))
- cf++;
- else
- noCf++;
- }
- else
- regular++;
+ ctx.update(row.key(), row.value(), row.version());
}
return new PartitionHashRecordV2(
partKey,
isPrimary,
consId,
- partHash,
- partVerHash,
updCntr,
partSize,
PartitionHashRecordV2.PartitionState.OWNING,
- cf,
- noCf,
- binary,
- regular
+ ctx
);
}
@@ -379,4 +354,64 @@ public IdleChecker(IgniteEx ig, Map(),
- GridIoPolicy.UNDEFINED,
- excHnd);
+ snpExecSvc = createExecutorService(
+ SNAPSHOT_RUNNER_THREAD_PREFIX,
+ cfg.getIgniteInstanceName(),
+ cfg.getSnapshotThreadPoolSize(),
+ cfg.getSnapshotThreadPoolSize(),
+ DFLT_THREAD_KEEP_ALIVE_TIME,
+ new LinkedBlockingQueue<>(),
+ GridIoPolicy.UNDEFINED,
+ excHnd);
- snpExecSvc.allowCoreThreadTimeOut(true);
+ snpExecSvc.allowCoreThreadTimeOut(true);
+ if (CU.isPersistenceEnabled(ctx.config())) {
reencryptExecSvc = createExecutorService(
"reencrypt",
ctx.igniteInstanceName(),
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/SnapshotView.java b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/SnapshotView.java
index a7078204bcd9b..06f582f423048 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/SnapshotView.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/systemview/view/SnapshotView.java
@@ -63,7 +63,7 @@ public SnapshotView(
SnapshotMetadata meta,
Collection cacheGrps
) {
- type = SnapshotType.FULL;
+ type = meta.dump() ? SnapshotType.DUMP : SnapshotType.FULL;
name = meta.snapshotName();
consistentId = meta.consistentId();
baselineNodes = F.concat(meta.baselineNodes(), ",");
@@ -149,6 +149,9 @@ private enum SnapshotType {
FULL,
/** Incremental snapshot. */
- INCREMENTAL
+ INCREMENTAL,
+
+ /** Dump. */
+ DUMP
}
}
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 802497a2d72b2..7890402f2ece8 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
@@ -824,7 +824,7 @@ protected static IgniteInternalFuture> startLocalSnapshotTask(
SnapshotSender snpSndr
) throws IgniteCheckedException {
AbstractSnapshotFutureTask> task = cctx.snapshotMgr().registerSnapshotTask(snpName, cctx.localNodeId(), null,
- parts, withMetaStorage, snpSndr);
+ parts, withMetaStorage, false, snpSndr);
if (!(task instanceof SnapshotFutureTask))
throw new IgniteCheckedException("Snapshot task hasn't been registered: " + task);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
index d16d6074f5267..d1d51093c21d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/EncryptedSnapshotTest.java
@@ -291,7 +291,7 @@ public void testSnapshotTaskIsBlockedWithoutMetastore() throws Exception {
GridTestUtils.assertThrowsAnyCause(log,
() -> snp(ig).registerSnapshotTask(SNAPSHOT_NAME, ig.localNode().id(),
- null, F.asMap(CU.cacheId(dfltCacheCfg.getName()), null), false,
+ null, F.asMap(CU.cacheId(dfltCacheCfg.getName()), null), false, false,
snp(ig).localSnapshotSenderFactory().apply(SNAPSHOT_NAME, null)).get(TIMEOUT),
IgniteCheckedException.class,
"Metastore is required because it holds encryption keys");
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
index 79b3699b6b0f5..478cbb750e25f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManagerSelfTest.java
@@ -148,6 +148,7 @@ public void testSnapshotLocalPartitionMultiCpWithLoad() throws Exception {
null,
F.asMap(CU.cacheId(DEFAULT_CACHE_NAME), null),
encryption,
+ false,
new DelegateSnapshotSender(log, mgr.snapshotExecutorService(), mgr.localSnapshotSenderFactory().apply(SNAPSHOT_NAME, null)) {
@Override public void sendPart0(File part, String cacheDirName, GroupPartitionId pair, Long length) {
try {
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/AbstractCacheDumpTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/AbstractCacheDumpTest.java
new file mode 100644
index 0000000000000..55b2974f76571
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/AbstractCacheDumpTest.java
@@ -0,0 +1,607 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.function.IntConsumer;
+import java.util.function.IntFunction;
+import java.util.stream.IntStream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cdc.TypeMapping;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.dump.DumpConsumer;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.dump.DumpReader;
+import org.apache.ignite.dump.DumpReaderConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.processors.cache.CacheGroupContext;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotMetadata;
+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.platform.model.ACL;
+import org.apache.ignite.platform.model.Key;
+import org.apache.ignite.platform.model.Role;
+import org.apache.ignite.platform.model.User;
+import org.apache.ignite.platform.model.Value;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.dump.DumpReaderConfiguration.DFLT_THREAD_CNT;
+import static org.apache.ignite.dump.DumpReaderConfiguration.DFLT_TIMEOUT;
+import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNP_RUNNING_DIR_KEY;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask.toLong;
+import static org.apache.ignite.platform.model.AccessLevel.SUPER;
+import static org.apache.ignite.testframework.GridTestUtils.runAsync;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+
+/** */
+@RunWith(Parameterized.class)
+public abstract class AbstractCacheDumpTest extends GridCommonAbstractTest {
+ /** */
+ public static final String GRP = "grp";
+
+ /** */
+ public static final String CACHE_0 = "cache-0";
+
+ /** */
+ public static final String CACHE_1 = "cache-1";
+
+ /** */
+ public static final int KEYS_CNT = 1000;
+
+ /** */
+ public static final String DMP_NAME = "dump";
+
+ /** */
+ public static final IntFunction USER_FACTORY = i ->
+ new User(i, ACL.values()[Math.abs(i) % ACL.values().length], new Role("Role" + i, SUPER));
+
+ /** */
+ @Parameterized.Parameter
+ public int nodes;
+
+ /** */
+ @Parameterized.Parameter(1)
+ public int backups;
+
+ /** */
+ @Parameterized.Parameter(2)
+ public boolean persistence;
+
+ /** */
+ @Parameterized.Parameter(3)
+ public CacheAtomicityMode mode;
+
+ /** */
+ @Parameterized.Parameter(4)
+ public boolean useDataStreamer;
+
+ /** */
+ @Parameterized.Parameter(5)
+ public boolean onlyPrimary;
+
+ /** */
+ @Parameterized.Parameters(name = "nodes={0},backups={1},persistence={2},mode={3},useDataStreamer={4},onlyPrimary={5}")
+ public static List params() {
+ List params = new ArrayList<>();
+
+ for (int nodes : new int[]{1, 3})
+ for (int backups : new int[]{0, 1})
+ for (boolean persistence : new boolean[]{true, false})
+ for (CacheAtomicityMode mode : CacheAtomicityMode.values()) {
+ for (boolean useDataStreamer : new boolean[]{true, false}) {
+ if (nodes == 1 && backups != 0)
+ continue;
+
+ if (backups > 0) {
+ params.add(new Object[]{nodes, backups, persistence, mode, useDataStreamer, false});
+ params.add(new Object[]{nodes, backups, persistence, mode, useDataStreamer, true});
+ }
+ else
+ params.add(new Object[]{nodes, backups, persistence, mode, useDataStreamer, false});
+ }
+ }
+
+ return params;
+ }
+
+ /** */
+ protected int snpPoolSz = 1;
+
+ /** */
+ protected IgniteEx cli;
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ super.beforeTest();
+
+ cleanPersistenceDir();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ super.afterTest();
+
+ stopAllGrids();
+ }
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ return super.getConfiguration(igniteInstanceName)
+ .setSnapshotThreadPoolSize(snpPoolSz)
+ .setDataStorageConfiguration(new DataStorageConfiguration()
+ .setDefaultDataRegionConfiguration(new DataRegionConfiguration().setPersistenceEnabled(persistence)))
+ .setCacheConfiguration(
+ new CacheConfiguration<>()
+ .setName(DEFAULT_CACHE_NAME)
+ .setBackups(backups)
+ .setAtomicityMode(mode)
+ .setWriteSynchronizationMode(FULL_SYNC)
+ .setAffinity(new RendezvousAffinityFunction().setPartitions(20)),
+ new CacheConfiguration<>()
+ .setGroupName(GRP)
+ .setName(CACHE_0)
+ .setBackups(backups)
+ .setAtomicityMode(mode)
+ .setWriteSynchronizationMode(FULL_SYNC)
+ .setAffinity(new RendezvousAffinityFunction().setPartitions(20)),
+ new CacheConfiguration<>()
+ .setGroupName(GRP)
+ .setName(CACHE_1)
+ .setBackups(backups)
+ .setAtomicityMode(mode)
+ .setWriteSynchronizationMode(FULL_SYNC)
+ .setAffinity(new RendezvousAffinityFunction().setPartitions(20))
+ );
+ }
+
+ /** */
+ protected IgniteEx startGridAndFillCaches() throws Exception {
+ IgniteEx ign = (IgniteEx)startGridsMultiThreaded(nodes);
+
+ cli = startClientGrid(nodes);
+
+ ign.cluster().state(ClusterState.ACTIVE);
+
+ putData(cli.cache(DEFAULT_CACHE_NAME), cli.cache(CACHE_0), cli.cache(CACHE_1));
+
+ return ign;
+ }
+
+ /** */
+ protected T2> runDumpAsyncAndStopBeforeStart() throws IgniteInterruptedCheckedException {
+ CountDownLatch latch = new CountDownLatch(1);
+
+ List ignites = Ignition.allGrids();
+
+ for (Ignite ign : ignites) {
+ ((IgniteEx)ign).context().pools().getSnapshotExecutorService().submit(() -> {
+ try {
+ latch.await();
+ }
+ catch (InterruptedException e) {
+ throw new IgniteException(e);
+ }
+ });
+ }
+
+ IgniteInternalFuture dumpFut = runAsync(() -> createDump((IgniteEx)F.first(ignites)));
+
+ // Waiting while dump will be setup: task planned after change listener set.
+ assertTrue(waitForCondition(() -> {
+ for (Ignite ign : ignites) {
+ if (ign.configuration().isClientMode() == Boolean.TRUE)
+ continue;
+
+ if (((ThreadPoolExecutor)((IgniteEx)ign).context().pools().getSnapshotExecutorService()).getTaskCount() <= 1)
+ return false;
+ }
+
+ return true;
+ }, 10 * 1000));
+
+ return new T2<>(latch, dumpFut);
+ }
+
+ /** */
+ protected void putData(
+ IgniteCache cache,
+ IgniteCache grpCache0,
+ IgniteCache grpCache1
+ ) {
+ if (useDataStreamer) {
+ try (
+ IgniteDataStreamer _cache = cli.dataStreamer(cache.getName());
+ IgniteDataStreamer _grpCache0 = cli.dataStreamer(grpCache0.getName());
+ IgniteDataStreamer _grpCache1 = cli.dataStreamer(grpCache1.getName())
+ ) {
+ IntStream.range(0, KEYS_CNT).forEach(i -> {
+ _cache.addData(i, i);
+ _grpCache0.addData(i, USER_FACTORY.apply(i));
+ _grpCache1.addData(new Key(i), new Value(String.valueOf(i)));
+ });
+ }
+ }
+ else {
+ IntStream.range(0, KEYS_CNT).forEach(i -> {
+ cache.put(i, i);
+ grpCache0.put(i, USER_FACTORY.apply(i));
+ grpCache1.put(new Key(i), new Value(String.valueOf(i)));
+ });
+ }
+ }
+
+ /** */
+ protected void checkDump(IgniteEx ign) throws Exception {
+ checkDump(ign, DMP_NAME);
+ }
+
+ /** */
+ void checkDump(IgniteEx ign, String name) throws Exception {
+ checkDumpWithCommand(ign, name, backups);
+
+ if (persistence)
+ assertNull(ign.context().cache().context().database().metaStorage().read(SNP_RUNNING_DIR_KEY));
+
+ Dump dump = dump(ign, name);
+
+ List metadata = dump.metadata();
+
+ assertNotNull(metadata);
+ assertEquals(nodes, metadata.size());
+
+ for (SnapshotMetadata meta : metadata) {
+ assertEquals(name, meta.snapshotName());
+ assertTrue(meta.dump());
+ assertFalse(meta.cacheGroupIds().contains(CU.cacheId(UTILITY_CACHE_NAME)));
+ }
+
+ List nodesDirs = dump.nodesDirectories();
+
+ assertEquals(nodes, nodesDirs.size());
+
+ TestDumpConsumer cnsmr = new TestDumpConsumer() {
+ final Set keys = new HashSet<>();
+
+ final Set grpParts = new HashSet<>();
+
+ int dfltDumpSz;
+
+ int grpDumpSz;
+
+ @Override public void onCacheConfigs(Iterator caches) {
+ super.onCacheConfigs(caches);
+
+ boolean[] cachesFound = new boolean[3];
+
+ caches.forEachRemaining(data -> {
+ if (data.config().getName().equals(DEFAULT_CACHE_NAME)) {
+ assertFalse(cachesFound[0]);
+ cachesFound[0] = true;
+
+ assertEquals(DEFAULT_CACHE_NAME, data.config().getName());
+ assertFalse(data.sql());
+ assertTrue(data.queryEntities().isEmpty());
+ }
+ else if (data.config().getName().equals(CACHE_0)) {
+ assertFalse(cachesFound[1]);
+ cachesFound[1] = true;
+
+ assertEquals(GRP, data.configuration().getGroupName());
+ assertEquals(CACHE_0, data.configuration().getName());
+ assertFalse(data.sql());
+ assertTrue(data.queryEntities().isEmpty());
+ }
+ else if (data.config().getName().equals(CACHE_1)) {
+ assertFalse(cachesFound[2]);
+ cachesFound[2] = true;
+
+ assertEquals(GRP, data.configuration().getGroupName());
+ assertEquals(CACHE_1, data.configuration().getName());
+ assertFalse(data.sql());
+ assertTrue(data.queryEntities().isEmpty());
+ }
+ else
+ throw new IgniteException("Unknown cache");
+ });
+
+ for (boolean found : cachesFound)
+ assertTrue(found);
+ }
+
+ @Override public void onPartition(int grp, int part, Iterator iter) {
+ if (onlyPrimary)
+ assertTrue(grpParts.add(toLong(grp, part)));
+
+ if (grp == CU.cacheId(DEFAULT_CACHE_NAME)) {
+ while (iter.hasNext()) {
+ DumpEntry e = iter.next();
+
+ checkDefaultCacheEntry(e);
+
+ keys.add((Integer)e.key());
+
+ dfltDumpSz++;
+ }
+ }
+ else {
+ while (iter.hasNext()) {
+ DumpEntry e = iter.next();
+
+ assertNotNull(e);
+
+ if (e.cacheId() == CU.cacheId(CACHE_0))
+ assertEquals(USER_FACTORY.apply((Integer)e.key()), e.value());
+ else
+ assertEquals(((Key)e.key()).getId() + "", ((Value)e.value()).getVal());
+
+ grpDumpSz++;
+ }
+ }
+ }
+
+ @Override public void check() {
+ super.check();
+
+ assertEquals(KEYS_CNT + (onlyPrimary ? 0 : KEYS_CNT * backups), dfltDumpSz);
+ assertEquals(2 * (KEYS_CNT + (onlyPrimary ? 0 : KEYS_CNT * backups)), grpDumpSz);
+
+ IntStream.range(0, KEYS_CNT).forEach(key -> assertTrue(keys.contains(key)));
+ }
+ };
+
+ new DumpReader(
+ new DumpReaderConfiguration(
+ dumpDirectory(ign, name),
+ cnsmr,
+ DFLT_THREAD_CNT, DFLT_TIMEOUT,
+ true,
+ false
+ ),
+ log
+ ).run();
+
+ cnsmr.check();
+ }
+
+ /** */
+ protected void checkDefaultCacheEntry(DumpEntry e) {
+ assertNotNull(e);
+
+ Integer key = (Integer)e.key();
+
+ assertEquals(key, e.value());
+ }
+
+ /** */
+ protected void insertOrUpdate(IgniteEx ignite, int i) {
+ insertOrUpdate(ignite, i, i);
+ }
+
+ /** */
+ protected void insertOrUpdate(IgniteEx ignite, int i, int val) {
+ ignite.cache(DEFAULT_CACHE_NAME).put(i, val);
+ IgniteCache cache = ignite.cache(CACHE_0);
+ IgniteCache cache1 = ignite.cache(CACHE_1);
+
+ if (mode == CacheAtomicityMode.TRANSACTIONAL) {
+ try (Transaction tx = ignite.transactions().txStart()) {
+ cache.put(i, USER_FACTORY.apply(val));
+
+ tx.commit();
+ }
+
+ try (Transaction tx = ignite.transactions().txStart()) {
+ cache1.put(new Key(i), new Value(String.valueOf(val)));
+
+ tx.commit();
+ }
+ }
+ else {
+ cache.put(i, USER_FACTORY.apply(val));
+
+ cache1.put(new Key(i), new Value(String.valueOf(val)));
+ }
+ }
+
+ /** */
+ protected void remove(IgniteEx ignite, int i) {
+ ignite.cache(DEFAULT_CACHE_NAME).remove(i);
+
+ IgniteCache cache = ignite.cache(CACHE_0);
+ IgniteCache cache1 = ignite.cache(CACHE_1);
+
+ IntConsumer moreRemovals = j -> {
+ cache.remove(j);
+ cache.remove(j);
+ cache1.remove(new Key(j));
+ cache1.remove(new Key(j));
+ };
+
+ if (mode == CacheAtomicityMode.TRANSACTIONAL) {
+ try (Transaction tx = ignite.transactions().txStart()) {
+ moreRemovals.accept(i);
+
+ tx.commit();
+ }
+ }
+ else
+ moreRemovals.accept(i);
+ }
+
+ /** */
+ protected void createDump(IgniteEx ign) {
+ createDump(ign, DMP_NAME);
+ }
+
+ /** */
+ public static Dump dump(IgniteEx ign, String name) throws IgniteCheckedException {
+ return new Dump(
+ dumpDirectory(ign, name),
+ true,
+ false,
+ log
+ );
+ }
+
+ /** */
+ public static File dumpDirectory(IgniteEx ign, String name) throws IgniteCheckedException {
+ return new File(U.resolveWorkDirectory(U.defaultWorkDirectory(), ign.configuration().getSnapshotPath(), false), name);
+ }
+
+ /** */
+ public static void checkDumpWithCommand(IgniteEx ign, String name, int backups) throws Exception {
+ CacheGroupContext gctx = ign.context().cache().cacheGroup(CU.cacheId(DEFAULT_CACHE_NAME));
+
+ for (GridCacheContext, ?> cctx : gctx.caches())
+ assertNull(cctx.dumpListener());
+
+ gctx = ign.context().cache().cacheGroup(CU.cacheId(GRP));
+
+ for (GridCacheContext, ?> cctx : gctx.caches())
+ assertNull(cctx.dumpListener());
+
+ assertEquals("The check procedure has finished, no conflicts have been found.\n\n", invokeCheckCommand(ign, name));
+ }
+
+ /** */
+ public static String invokeCheckCommand(IgniteEx ign, String name) throws IgniteCheckedException {
+ StringBuffer buf = new StringBuffer();
+
+ ign.context().cache().context().snapshotMgr().checkSnapshot(name, null).get(60_000)
+ .print(line -> buf.append(line).append(System.lineSeparator()));
+
+ return buf.toString();
+ }
+
+ /** */
+ void createDump(IgniteEx ign, String name) {
+ ign.context().cache().context().snapshotMgr().createSnapshot(name, null, false, onlyPrimary, true).get();
+ }
+
+ /** */
+ public abstract static class TestDumpConsumer implements DumpConsumer {
+ /** */
+ private boolean started;
+
+ /** */
+ private boolean stopped;
+
+ /** */
+ private boolean typesCb;
+
+ /** */
+ private boolean mappingcCb;
+
+ /** */
+ private boolean cacheCfgCb;
+
+ /** {@inheritDoc} */
+ @Override public void start() {
+ assertFalse(started);
+ assertFalse(mappingcCb);
+ assertFalse(typesCb);
+ assertFalse(cacheCfgCb);
+ assertFalse(stopped);
+
+ started = true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onMappings(Iterator mappings) {
+ assertTrue(started);
+ assertFalse(mappingcCb);
+ assertFalse(typesCb);
+ assertFalse(cacheCfgCb);
+ assertFalse(stopped);
+
+ mappingcCb = true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onTypes(Iterator types) {
+ assertTrue(started);
+ assertTrue(mappingcCb);
+ assertFalse(typesCb);
+ assertFalse(cacheCfgCb);
+ assertFalse(stopped);
+
+ typesCb = true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onCacheConfigs(Iterator caches) {
+ assertTrue(started);
+ assertTrue(mappingcCb);
+ assertTrue(typesCb);
+ assertFalse(cacheCfgCb);
+ assertFalse(stopped);
+
+ cacheCfgCb = true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void stop() {
+ assertTrue(started);
+ assertTrue(typesCb);
+ assertTrue(mappingcCb);
+ assertTrue(cacheCfgCb);
+ assertFalse(stopped);
+
+ stopped = true;
+ }
+
+ /** */
+ public void check() {
+ assertTrue(started);
+ assertTrue(typesCb);
+ assertTrue(mappingcCb);
+ assertTrue(cacheCfgCb);
+ assertTrue(stopped);
+ }
+ }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelf2Test.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelf2Test.java
new file mode 100644
index 0000000000000..e045298263ce7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelf2Test.java
@@ -0,0 +1,328 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+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.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersionManager;
+import org.apache.ignite.internal.processors.cacheobject.UserCacheObjectImpl;
+import org.apache.ignite.internal.processors.dr.GridDrType;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.platform.model.User;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static java.nio.file.StandardOpenOption.READ;
+import static java.nio.file.StandardOpenOption.WRITE;
+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.PART_FILE_PREFIX;
+import static org.apache.ignite.internal.processors.cache.persistence.filename.PdsFolderResolver.DB_DEFAULT_FOLDER;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.DUMP_LOCK;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.DMP_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.KEYS_CNT;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.USER_FACTORY;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.dump;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.invokeCheckCommand;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask.DUMP_FILE_EXT;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.DumpEntrySerializer.HEADER_SZ;
+import static org.apache.ignite.testframework.GridTestUtils.assertContains;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+
+/** */
+public class IgniteCacheDumpSelf2Test extends GridCommonAbstractTest {
+ /** */
+ private LogListener lsnr;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ if (lsnr != null) {
+ ListeningTestLogger testLog = new ListeningTestLogger(log);
+
+ testLog.registerListener(lsnr);
+
+ cfg.setGridLogger(testLog);
+ }
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ super.beforeTest();
+
+ stopAllGrids();
+
+ cleanPersistenceDir();
+ }
+
+ /** */
+ @Test
+ public void testDumpFailIfNoCaches() throws Exception {
+ try (IgniteEx ign = startGrid(new IgniteConfiguration())) {
+ ign.cluster().state(ClusterState.ACTIVE);
+
+ assertThrows(
+ null,
+ () -> ign.snapshot().createDump("dump").get(),
+ IgniteException.class,
+ "Dump operation has been rejected. No cache group defined in cluster"
+ );
+ }
+ }
+
+ /** */
+ @Test
+ public void testUnreadyDumpCleared() throws Exception {
+ IgniteEx ign = (IgniteEx)startGridsMultiThreaded(2);
+
+ ign.cluster().state(ClusterState.ACTIVE);
+
+ IgniteCache cache = ign.createCache(DEFAULT_CACHE_NAME);
+
+ IntStream.range(0, KEYS_CNT).forEach(i -> cache.put(i, i));
+
+ ign.snapshot().createDump(DMP_NAME).get(getTestTimeout());
+
+ stopAllGrids();
+
+ Dump dump = dump(ign, DMP_NAME);
+
+ List nodes = dump.nodesDirectories();
+
+ assertNotNull(nodes);
+ assertEquals(2, nodes.size());
+
+ File nodeDumpDir = new File(dump.dumpDirectory(), DB_DEFAULT_FOLDER + File.separator + nodes.get(0));
+
+ assertTrue(new File(nodeDumpDir, DUMP_LOCK).createNewFile());
+
+ lsnr = LogListener.matches("Found locked dump dir. " +
+ "This means, dump creation not finished prior to node fail. " +
+ "Directory will be deleted: " + nodeDumpDir.getAbsolutePath()).build();
+
+ startGridsMultiThreaded(2);
+
+ assertFalse(nodeDumpDir.exists());
+ assertTrue(lsnr.check());
+ }
+
+ /** */
+ @Test
+ public void testDumpIteratorFaileOnWrongCrc() throws Exception {
+ try (IgniteEx ign = startGrid(new IgniteConfiguration())) {
+ ign.cluster().state(ClusterState.ACTIVE);
+
+ IgniteCache cache = ign.createCache(DEFAULT_CACHE_NAME);
+
+ for (int key : partitionKeys(cache, 0, KEYS_CNT, 0))
+ cache.put(key, key);
+
+ ign.snapshot().createDump(DMP_NAME).get();
+
+ Dump dump = dump(ign, DMP_NAME);
+
+ List nodes = dump.nodesDirectories();
+
+ assertNotNull(nodes);
+ assertEquals(1, nodes.size());
+
+ File cacheDumpDir = new File(
+ dump.dumpDirectory(),
+ DB_DEFAULT_FOLDER + File.separator + nodes.get(0) + File.separator + CACHE_DIR_PREFIX + DEFAULT_CACHE_NAME
+ );
+
+ assertTrue(cacheDumpDir.exists());
+
+ Set dumpFiles = Arrays.asList(cacheDumpDir.listFiles()).stream()
+ .filter(f -> {
+ try {
+ return Files.size(f.toPath()) > 0;
+ }
+ catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ })
+ .collect(Collectors.toSet());
+
+ String partDumpName = PART_FILE_PREFIX + 0 + DUMP_FILE_EXT;
+
+ assertTrue(dumpFiles.stream().anyMatch(f -> f.getName().equals(CACHE_DATA_FILENAME)));
+ assertTrue(dumpFiles.stream().anyMatch(f -> f.getName().equals(partDumpName)));
+
+ try (FileChannel fc = FileChannel.open(Paths.get(cacheDumpDir.getAbsolutePath(), partDumpName), READ, WRITE)) {
+ fc.position(HEADER_SZ); // Skip first entry header.
+
+ int bufSz = 5;
+
+ ByteBuffer buf = ByteBuffer.allocate(bufSz);
+
+ assertEquals(bufSz, fc.read(buf));
+
+ buf.position(0);
+
+ // Increment first five bytes in dumped entry.
+ for (int i = 0; i < bufSz; i++) {
+ byte b = buf.get();
+ b++;
+ buf.position(i);
+ buf.put(b);
+ }
+
+ fc.position(HEADER_SZ);
+
+ buf.rewind();
+ fc.write(buf);
+ }
+
+ assertThrows(
+ null,
+ () -> dump.iterator(nodes.get(0), CU.cacheId(DEFAULT_CACHE_NAME), 0).next(),
+ IgniteException.class,
+ "Data corrupted"
+ );
+ }
+ }
+
+ /** */
+ @Test
+ public void testCheckFailOnCorruptedData() throws Exception {
+ IgniteEx ign = (IgniteEx)startGridsMultiThreaded(2);
+
+ IgniteCache cache = ign.createCache(new CacheConfiguration()
+ .setName(DEFAULT_CACHE_NAME)
+ .setBackups(1)
+ .setAtomicityMode(CacheAtomicityMode.ATOMIC));
+
+ IntStream.range(0, KEYS_CNT).forEach(i -> cache.put(i, i));
+
+ int corruptedPart = 1;
+ int corruptedKey = partitionKeys(cache, corruptedPart, 1, 0).get(0);
+
+ cache.put(corruptedKey, corruptedKey);
+
+ IgniteInternalCache cachex = ign.cachex(DEFAULT_CACHE_NAME);
+
+ GridCacheVersionManager mgr = cachex.context().shared().versions();
+
+ GridCacheAdapter adapter = (GridCacheAdapter)cachex.cache();
+
+ GridCacheEntryEx entry = adapter.entryEx(corruptedKey);
+
+ entry.innerUpdate(
+ mgr.next(entry.context().kernalContext().discovery().topologyVersion()),
+ ign.localNode().id(),
+ ign.localNode().id(),
+ GridCacheOperation.UPDATE,
+ new UserCacheObjectImpl(corruptedKey + 1, null),
+ null,
+ false,
+ false,
+ false,
+ false,
+ null,
+ false,
+ false,
+ false,
+ false,
+ false,
+ AffinityTopologyVersion.NONE,
+ null,
+ GridDrType.DR_NONE,
+ 0,
+ 0,
+ null,
+ false,
+ false,
+ null,
+ null,
+ null,
+ null,
+ false);
+
+ ign.snapshot().createDump(DMP_NAME).get();
+
+ assertContains(
+ null,
+ invokeCheckCommand(ign, DMP_NAME),
+ "Conflict partition: PartitionKeyV2 [grpId=" + CU.cacheId(DEFAULT_CACHE_NAME) +
+ ", grpName=" + DEFAULT_CACHE_NAME +
+ ", partId=" + corruptedPart + "]"
+ );
+ }
+
+ /** */
+ @Test
+ public void testCheckOnEmptyNode() throws Exception {
+ String id = "test";
+
+ IgniteEx ign = startGrid(getConfiguration(id).setConsistentId(id));
+
+ IgniteCache cache = ign.createCache(new CacheConfiguration()
+ .setName("test-cache-0")
+ .setBackups(1)
+ .setAtomicityMode(CacheAtomicityMode.ATOMIC));
+
+ IgniteCache cache2 = ign.createCache(new CacheConfiguration()
+ .setName("users")
+ .setBackups(1)
+ .setAtomicityMode(CacheAtomicityMode.ATOMIC));
+
+ IntStream.range(0, KEYS_CNT).forEach(i -> {
+ cache.put(i, i);
+ cache2.put(i, USER_FACTORY.apply(i));
+ });
+
+ ign.snapshot().createDump(DMP_NAME).get();
+
+ assertEquals("The check procedure has finished, no conflicts have been found.\n\n", invokeCheckCommand(ign, DMP_NAME));
+
+ stopAllGrids();
+
+ cleanPersistenceDir(true);
+
+ ign = startGrid(getConfiguration(id).setConsistentId(id));
+
+ assertEquals("The check procedure has finished, no conflicts have been found.\n\n", invokeCheckCommand(ign, DMP_NAME));
+ }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelfTest.java
new file mode 100644
index 0000000000000..9491a3f65340c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteCacheDumpSelfTest.java
@@ -0,0 +1,506 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.OpenOption;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.function.Consumer;
+import java.util.stream.IntStream;
+import javax.cache.expiry.Duration;
+import javax.cache.expiry.ExpiryPolicy;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIODecorator;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIOFactory;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.platform.model.Key;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static java.lang.Boolean.FALSE;
+import static java.lang.Boolean.TRUE;
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotManager.SNP_RUNNING_DIR_KEY;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.CreateDumpFutureTask.DUMP_FILE_EXT;
+import static org.apache.ignite.testframework.GridTestUtils.assertThrows;
+import static org.junit.Assume.assumeFalse;
+
+/** */
+public class IgniteCacheDumpSelfTest extends AbstractCacheDumpTest {
+ /** */
+ public static final String EXISTS_ERR_MSG = "Create snapshot request has been rejected. " +
+ "Snapshot with given name already exists on local node";
+
+ /** */
+ public static final long TTL = 5 * 1000;
+
+ /** */
+ public static final ExpiryPolicy EXPIRY_POLICY = new ExpiryPolicy() {
+ @Override public Duration getExpiryForCreation() {
+ return new Duration(MILLISECONDS, TTL);
+ }
+
+ @Override public Duration getExpiryForAccess() {
+ return null;
+ }
+
+ @Override public Duration getExpiryForUpdate() {
+ return null;
+ }
+ };
+
+ /** */
+ private Boolean explicitTtl;
+
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ super.beforeTest();
+
+ explicitTtl = null;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ if (explicitTtl == TRUE) {
+ for (CacheConfiguration, ?> ccfg : cfg.getCacheConfiguration())
+ ccfg.setExpiryPolicyFactory(() -> EXPIRY_POLICY);
+ }
+
+ return cfg;
+ }
+
+ /** */
+ @Test
+ public void testCacheDump() throws Exception {
+ snpPoolSz = 4;
+
+ try {
+ IgniteEx ign = startGridAndFillCaches();
+
+ createDump(ign);
+
+ checkDump(ign);
+
+ assertThrows(null, () -> createDump(ign), IgniteException.class, EXISTS_ERR_MSG);
+
+ createDump(ign, DMP_NAME + 2);
+
+ checkDump(ign, DMP_NAME + 2);
+
+ if (persistence) {
+ assertThrows(null, () -> ign.snapshot().createSnapshot(DMP_NAME).get(), IgniteException.class, EXISTS_ERR_MSG);
+
+ ign.snapshot().createSnapshot(DMP_NAME + 3).get();
+ }
+ else {
+ assertThrows(
+ null,
+ () -> ign.snapshot().createSnapshot(DMP_NAME + 3).get(),
+ IgniteException.class,
+ "Create snapshot request has been rejected. Snapshots on an in-memory clusters are not allowed."
+ );
+ }
+ }
+ finally {
+ snpPoolSz = 1;
+ }
+ }
+
+ /** */
+ @Test
+ public void testConcurrentDumpCreationThrows() throws Exception {
+ doTestConcurrentOperations(ignite -> {
+ assertThrows(
+ null,
+ () -> createDump(ignite, "other_dump"),
+ IgniteException.class,
+ "Create snapshot request has been rejected. The previous snapshot operation was not completed."
+ );
+
+ if (persistence) {
+ assertThrows(
+ null,
+ () -> ignite.snapshot().createSnapshot("other_snapshot").get(getTestTimeout()),
+ IgniteException.class,
+ "Create snapshot request has been rejected. The previous snapshot operation was not completed."
+ );
+ }
+ });
+ }
+
+ /** */
+ @Test
+ public void testWithConcurrentInserts() throws Exception {
+ doTestConcurrentOperations(ignite -> {
+ for (int i = KEYS_CNT * 10; i < KEYS_CNT * 10 + 3; i++) {
+ assertFalse(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+ assertFalse(ignite.cache(CACHE_0).containsKey(i));
+ assertFalse(ignite.cache(CACHE_1).containsKey(new Key(i)));
+
+ insertOrUpdate(ignite, i);
+ }
+
+ for (int i = KEYS_CNT * 10 + 3; i < KEYS_CNT * 10 + 6; i++) {
+ assertFalse(cli.cache(DEFAULT_CACHE_NAME).containsKey(i));
+ assertFalse(cli.cache(CACHE_0).containsKey(i));
+ assertFalse(cli.cache(CACHE_1).containsKey(new Key(i)));
+
+ insertOrUpdate(cli, i);
+ }
+ });
+ }
+
+ /** */
+ @Test
+ public void testWithConcurrentUpdates() throws Exception {
+ doTestConcurrentOperations(ignite -> {
+ for (int i = 0; i < 3; i++) {
+ assertTrue(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+ assertTrue(ignite.cache(CACHE_0).containsKey(i));
+ assertTrue(ignite.cache(CACHE_1).containsKey(new Key(i)));
+
+ insertOrUpdate(ignite, i);
+ }
+
+ for (int i = 3; i < 6; i++) {
+ assertTrue(cli.cache(DEFAULT_CACHE_NAME).containsKey(i));
+ assertTrue(cli.cache(CACHE_0).containsKey(i));
+ assertTrue(cli.cache(CACHE_1).containsKey(new Key(i)));
+
+ insertOrUpdate(cli, i);
+ }
+ });
+ }
+
+ /** */
+ @Test
+ public void testWithConcurrentEntryProcessor() throws Exception {
+ doTestConcurrentOperations(ignite -> {
+ for (int i = KEYS_CNT * 10; i < KEYS_CNT * 10 + 3; i++) {
+ assertFalse(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+
+ ignite.cache(DEFAULT_CACHE_NAME).invoke(i, new CacheEntryProcessor() {
+ @Override public Void process(MutableEntry entry, Object... arguments) {
+ entry.setValue(entry.getKey());
+ return null;
+ }
+ });
+ }
+
+ for (int i = 0; i < 3; i++) {
+ assertTrue(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+
+ ignite.cache(DEFAULT_CACHE_NAME).invoke(i, new CacheEntryProcessor() {
+ @Override public Void process(MutableEntry entry, Object... arguments) {
+ entry.setValue(entry.getKey() + 1);
+ return null;
+ }
+ });
+ }
+ });
+ }
+
+ /** */
+ @Test
+ public void testWithConcurrentRemovals() throws Exception {
+ doTestConcurrentOperations(ignite -> {
+ for (int i = 0; i < 3; i++) {
+ assertTrue(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+
+ remove(ignite, i);
+ }
+
+ for (int i = 3; i < 6; i++) {
+ assertTrue(ignite.cache(DEFAULT_CACHE_NAME).containsKey(i));
+
+ remove(cli, i);
+ }
+ });
+ }
+
+ /** */
+ @Test
+ public void testDumpWithExplicitExpireTime() throws Exception {
+ explicitTtl = true;
+
+ doTestDumpWithExpiry();
+ }
+
+ /** */
+ @Test
+ public void testDumpWithImplicitExpireTime() throws Exception {
+ assumeFalse(useDataStreamer);
+
+ explicitTtl = false;
+
+ doTestDumpWithExpiry();
+ }
+
+ /** */
+ @Test
+ public void testDumpCancelOnFileCreateError() throws Exception {
+ IgniteEx ign = startGridAndFillCaches();
+
+ for (Ignite node : G.allGrids()) {
+ if (node.configuration().isClientMode() == TRUE)
+ continue;
+
+ ((IgniteEx)node).context().cache().context().snapshotMgr().ioFactory(new DumpFailingFactory((IgniteEx)node, false));
+ }
+
+ assertThrows(null, () -> ign.snapshot().createDump(DMP_NAME).get(), IgniteException.class, "Test error");
+
+ checkDumpCleared(ign);
+
+ ign.cache(DEFAULT_CACHE_NAME).put(KEYS_CNT, KEYS_CNT);
+ }
+
+ /** */
+ @Test
+ public void testDumpCancelOnIteratorWriteError() throws Exception {
+ IgniteEx ign = startGridAndFillCaches();
+
+ DumpFailingFactory ioFactory = new DumpFailingFactory(ign, true);
+
+ ign.context().cache().context().snapshotMgr().ioFactory(ioFactory);
+
+ assertThrows(null, () -> ign.snapshot().createDump(DMP_NAME).get(), IgniteException.class, "Test write error");
+
+ assertTrue(ioFactory.errorAfter.get() <= 0);
+
+ checkDumpCleared(ign);
+
+ ign.cache(DEFAULT_CACHE_NAME).put(KEYS_CNT, KEYS_CNT);
+ }
+
+ /** */
+ @Test
+ public void testDumpCancelOnListenerWriteError() throws Exception {
+ IgniteEx ign = startGridAndFillCaches();
+
+ IgniteCache cache = ign.cache(DEFAULT_CACHE_NAME);
+
+ int keyToFail = findKeys(ign.localNode(), cache, 1, KEYS_CNT, 0).get(0);
+
+ byte[] valToFail = new byte[100];
+
+ AtomicBoolean keyToFailFound = new AtomicBoolean();
+
+ IntStream.range(0, 100).forEach(i -> valToFail[i] = (byte)i);
+
+ cache.put(keyToFail, valToFail);
+
+ FileIOFactory delegate = ign.context().cache().context().snapshotMgr().ioFactory();
+
+ ign.context().cache().context().snapshotMgr().ioFactory(new FileIOFactory() {
+ @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+ if (file.getName().endsWith(DUMP_FILE_EXT)) {
+ return new FileIODecorator(delegate.create(file, modes)) {
+ /** {@inheritDoc} */
+ @Override public int writeFully(ByteBuffer srcBuf) throws IOException {
+ if (findValToFail(srcBuf)) {
+ keyToFailFound.set(true);
+
+ throw new IOException("Val to fail found");
+ }
+
+ return super.writeFully(srcBuf);
+ }
+
+ private boolean findValToFail(ByteBuffer srcBuf) {
+ if (srcBuf.limit() >= valToFail.length) {
+ int valIdx = 0;
+
+ for (int i = 0; i < srcBuf.limit(); i++) {
+ if (srcBuf.get(i) == valToFail[valIdx]) {
+ valIdx++;
+
+ if (valIdx == valToFail.length)
+ return true;
+ }
+ else
+ valIdx = 0;
+ }
+ }
+
+ return false;
+ }
+ };
+ }
+
+ return delegate.create(file, modes);
+ }
+ });
+
+ T2> latchAndFut = runDumpAsyncAndStopBeforeStart();
+
+ cache.put(keyToFail, "test string");
+
+ latchAndFut.get1().countDown();
+
+ assertThrows(null, () -> latchAndFut.get2().get(10 * 1000), IgniteCheckedException.class, "Val to fail found");
+
+ assertTrue(keyToFailFound.get());
+
+ checkDumpCleared(ign);
+
+ cache.put(keyToFail, valToFail);
+ }
+
+ /** */
+ private void checkDumpCleared(IgniteEx ign) throws IgniteCheckedException {
+ if (persistence)
+ assertNull(ign.context().cache().context().database().metaStorage().read(SNP_RUNNING_DIR_KEY));
+
+ assertFalse(
+ new File(U.resolveWorkDirectory(U.defaultWorkDirectory(), ign.configuration().getSnapshotPath(), false), DMP_NAME).exists()
+ );
+ }
+
+ /** */
+ private void doTestDumpWithExpiry() throws Exception {
+ IgniteEx ign = startGridAndFillCaches();
+
+ T2> latchAndFut = runDumpAsyncAndStopBeforeStart();
+
+ Thread.sleep(TTL);
+
+ assertTrue(GridTestUtils.waitForCondition(() -> {
+ for (int i = 0; i < KEYS_CNT; i++) {
+ if (ign.cache(DEFAULT_CACHE_NAME).containsKey(i))
+ return false;
+
+ if (ign.cache(CACHE_0).containsKey(i))
+ return false;
+
+ if (ign.cache(CACHE_1).containsKey(new Key(i)))
+ return false;
+ }
+
+ return true;
+ }, 2 * TTL));
+
+ latchAndFut.get1().countDown();
+
+ latchAndFut.get2().get();
+
+ checkDump(ign);
+ }
+
+ /** */
+ private void doTestConcurrentOperations(Consumer op) throws Exception {
+ IgniteEx ign = startGridAndFillCaches();
+
+ T2> latchAndFut = runDumpAsyncAndStopBeforeStart();
+
+ // This operations will be catched by change listeners. Old value must be stored in dump.
+ op.accept(ign);
+
+ latchAndFut.get1().countDown();
+
+ latchAndFut.get2().get(10 * 1000);
+
+ checkDump(ign);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void putData(
+ IgniteCache cache,
+ IgniteCache grpCache0,
+ IgniteCache grpCache1
+ ) {
+ if (explicitTtl == FALSE) {
+ super.putData(
+ cache.withExpiryPolicy(EXPIRY_POLICY),
+ grpCache0.withExpiryPolicy(EXPIRY_POLICY),
+ grpCache1.withExpiryPolicy(EXPIRY_POLICY)
+ );
+ }
+ else
+ super.putData(cache, grpCache0, grpCache1);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void checkDefaultCacheEntry(DumpEntry e) {
+ super.checkDefaultCacheEntry(e);
+
+ if (explicitTtl != null) {
+ assertTrue("Expire time must be set", e.expireTime() != 0);
+ assertTrue("Expire time must be in past", System.currentTimeMillis() >= e.expireTime());
+ assertTrue("Expire time must be set during test run", System.currentTimeMillis() - getTestTimeout() < e.expireTime());
+ }
+ }
+
+ /** */
+ public static class DumpFailingFactory implements FileIOFactory {
+ /** */
+ private final FileIOFactory delegate;
+
+ /** */
+ private final AtomicInteger errorAfter;
+
+ /** */
+ private final boolean failOnWrite;
+
+ /** */
+ public DumpFailingFactory(IgniteEx ign, boolean failOnWrite) {
+ this.delegate = ign.context().cache().context().snapshotMgr().ioFactory();
+ this.failOnWrite = failOnWrite;
+ this.errorAfter = new AtomicInteger(KEYS_CNT / 20);
+ }
+
+ /** {@inheritDoc} */
+ @Override public FileIO create(File file, OpenOption... modes) throws IOException {
+ if (failOnWrite) {
+ return new FileIODecorator(delegate.create(file, modes)) {
+ /** {@inheritDoc} */
+ @Override public int writeFully(ByteBuffer srcBuf) throws IOException {
+ if (errorAfter.decrementAndGet() > 0)
+ return super.writeFully(srcBuf);
+
+ throw new IOException("Test write error");
+ }
+ };
+ }
+ else if (file.getName().endsWith(DUMP_FILE_EXT)) {
+ throw new IOException("Test error");
+ }
+
+ return delegate.create(file, modes);
+ }
+ }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteConcurrentCacheDumpTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteConcurrentCacheDumpTest.java
new file mode 100644
index 0000000000000..4e2a607cf1732
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/dump/IgniteConcurrentCacheDumpTest.java
@@ -0,0 +1,123 @@
+/*
+ * 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.dump;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.IntStream;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+
+/** */
+public class IgniteConcurrentCacheDumpTest extends AbstractCacheDumpTest {
+ /** */
+ @Parameterized.Parameters(name = "nodes={0},backups={1},persistence={2},mode={3},useDataStreamer={4},onlyPrimary={5}")
+ public static List params() {
+ List params = new ArrayList<>();
+
+ for (int nodes : new int[]{2, 3})
+ for (int backups : new int[]{1, 2})
+ for (boolean persistence : new boolean[]{true, false})
+ for (CacheAtomicityMode mode : CacheAtomicityMode.values())
+ params.add(new Object[]{nodes, backups, persistence, mode, false, false});
+
+ return params;
+ }
+
+ /** */
+ @Test
+ public void testDumpWithConcurrentOperations() throws Exception {
+ int thCnt = 5;
+
+ CountDownLatch initLatch = new CountDownLatch(thCnt);
+ AtomicBoolean canceled = new AtomicBoolean(false);
+
+ try (IgniteEx srv = (IgniteEx)startGridsMultiThreaded(nodes)) {
+ IgniteInternalFuture opFut = GridTestUtils.runMultiThreadedAsync(() -> {
+ Set keys = new TreeSet<>();
+
+ ThreadLocalRandom rand = ThreadLocalRandom.current();
+
+ IntStream.range(0, KEYS_CNT).forEach(i -> {
+ int k = rand.nextInt(KEYS_CNT);
+
+ keys.add(k);
+
+ insertOrUpdate(srv, i);
+ });
+
+ initLatch.countDown();
+
+ while (!canceled.get()) {
+ switch (rand.nextInt(3)) {
+ case 0:
+ int newKey = rand.nextInt(KEYS_CNT);
+
+ int iter = 0;
+
+ while (keys.contains(newKey) && iter < 10) {
+ newKey = rand.nextInt(KEYS_CNT);
+ iter++;
+ }
+
+ keys.add(newKey);
+
+ insertOrUpdate(srv, newKey, rand.nextInt());
+
+ break;
+ case 1:
+ int updKey = keys.isEmpty() ? rand.nextInt(KEYS_CNT) : keys.iterator().next();
+
+ insertOrUpdate(srv, updKey, rand.nextInt());
+
+ break;
+ case 2:
+ if (keys.isEmpty())
+ break;
+
+ int rmvKey = keys.isEmpty() ? rand.nextInt(KEYS_CNT) : keys.iterator().next();
+
+ keys.remove(rmvKey);
+
+ remove(srv, rmvKey);
+ }
+ }
+ }, thCnt, "dump-async-ops");
+
+ assertTrue(initLatch.await(30, TimeUnit.SECONDS));
+
+ createDump(srv, DMP_NAME);
+
+ canceled.set(true);
+
+ opFut.get();
+
+ checkDumpWithCommand(srv, DMP_NAME, backups);
+ }
+ }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/model/Key.java b/modules/core/src/test/java/org/apache/ignite/platform/model/Key.java
index 218d9c0b1e1e8..70caefc1615a5 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/model/Key.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/model/Key.java
@@ -48,4 +48,9 @@ public long getId() {
@Override public int hashCode() {
return Objects.hash(id);
}
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return "Key[id=" + id + ']';
+ }
}
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/model/Role.java b/modules/core/src/test/java/org/apache/ignite/platform/model/Role.java
index ad8b3d3f90885..a39fc2471ec4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/model/Role.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/model/Role.java
@@ -17,6 +17,8 @@
package org.apache.ignite.platform.model;
+import java.util.Objects;
+
/** Test value object. */
public class Role {
/** */
@@ -50,4 +52,17 @@ public AccessLevel getAccessLevel() {
public void setAccessLevel(AccessLevel accessLevel) {
this.accessLevel = accessLevel;
}
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ Role role = (Role)o;
+ return Objects.equals(name, role.name) && accessLevel == role.accessLevel;
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ return Objects.hash(name, accessLevel);
+ }
}
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/model/User.java b/modules/core/src/test/java/org/apache/ignite/platform/model/User.java
index 88849d16d658c..3377765be7629 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/model/User.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/model/User.java
@@ -17,6 +17,8 @@
package org.apache.ignite.platform.model;
+import java.util.Objects;
+
/** Test value object. */
public class User {
/** */
@@ -64,4 +66,17 @@ public Role getRole() {
public void setRole(Role role) {
this.role = role;
}
+
+ /** {@inheritDoc} */
+ @Override public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+ User user = (User)o;
+ return id == user.id && acl == user.acl && Objects.equals(role, user.role);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int hashCode() {
+ return Objects.hash(id, acl, role);
+ }
}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 1865af069b90a..30eb29cde09f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -40,6 +40,8 @@
import org.apache.ignite.spi.metric.noop.NoopMetricExporterSpi;
import org.apache.ignite.testframework.GridTestUtils;
+import static org.apache.ignite.internal.processors.cache.persistence.wal.reader.StandaloneGridKernalContext.startAllComponents;
+
/**
* Test context.
*/
@@ -94,8 +96,7 @@ public GridTestKernalContext(IgniteLogger log, IgniteConfiguration cfg) {
* @throws IgniteCheckedException If failed
*/
public void start() throws IgniteCheckedException {
- for (GridComponent comp : this)
- comp.start();
+ startAllComponents(this);
}
/**
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSnapshotTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSnapshotTestSuite.java
index a8a9ed93d31d6..ced347b5c0009 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSnapshotTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSnapshotTestSuite.java
@@ -35,6 +35,9 @@
import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotRestoreFromRemoteTest;
import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteSnapshotWithMetastorageTest;
import org.apache.ignite.internal.processors.cache.persistence.snapshot.PlainSnapshotTest;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.IgniteCacheDumpSelf2Test;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.IgniteCacheDumpSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.IgniteConcurrentCacheDumpTest;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.DynamicSuite;
import org.junit.runner.RunWith;
@@ -69,5 +72,8 @@ public static void addSnapshotTests(List> suite, Collection igno
GridTestUtils.addTestIfNeeded(suite, IgniteSnapshotConsistencyTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, IgniteClusterSnapshotDeltaTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, IncrementalSnapshotsTestSuite.class, ignoredTests);
+ GridTestUtils.addTestIfNeeded(suite, IgniteCacheDumpSelfTest.class, ignoredTests);
+ GridTestUtils.addTestIfNeeded(suite, IgniteCacheDumpSelf2Test.class, ignoredTests);
+ GridTestUtils.addTestIfNeeded(suite, IgniteConcurrentCacheDumpTest.class, ignoredTests);
}
}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java
index 0cd6b7964ce0a..8691a0a91d28b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/cdc/SqlCdcTest.java
@@ -376,7 +376,7 @@ else if (evt.configuration().getName().equals(USER)) {
}
/** */
- static List> executeSql(IgniteEx node, String sqlText, Object... args) {
+ public static List> executeSql(IgniteEx node, String sqlText, Object... args) {
return node.context().query().querySqlFields(new SqlFieldsQuery(sqlText).setArgs(args), true).getAll();
}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/dump/DumpCacheConfigTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/dump/DumpCacheConfigTest.java
new file mode 100644
index 0000000000000..18a24fbd63b9f
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/dump/DumpCacheConfigTest.java
@@ -0,0 +1,149 @@
+/*
+ * 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.dump;
+
+import java.io.File;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.dump.DumpEntry;
+import org.apache.ignite.dump.DumpReader;
+import org.apache.ignite.dump.DumpReaderConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.StoredCacheData;
+import org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.TestDumpConsumer;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.cdc.SqlCdcTest.executeSql;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.DMP_NAME;
+import static org.apache.ignite.internal.processors.cache.persistence.snapshot.dump.AbstractCacheDumpTest.KEYS_CNT;
+
+/** */
+public class DumpCacheConfigTest extends GridCommonAbstractTest {
+ /** {@inheritDoc} */
+ @Override protected void beforeTest() throws Exception {
+ cleanPersistenceDir();
+
+ super.beforeTest();
+ }
+
+ /** */
+ @Test
+ public void testSQLTableDump() throws Exception {
+ IgniteEx srv = (IgniteEx)startGridsMultiThreaded(2);
+
+ executeSql(srv, "CREATE TABLE T1(ID INT, NAME VARCHAR, PRIMARY KEY (ID)) WITH \"CACHE_NAME=T1\"");
+
+ for (int i = 0; i < KEYS_CNT; i++)
+ executeSql(srv, "INSERT INTO T1 VALUES(?, ?)", i, "Name-" + i);
+
+ checkDump(srv, DMP_NAME, true);
+
+ executeSql(srv, "ALTER TABLE T1 ADD COLUMN(ADDRESS VARCHAR)");
+
+ for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
+ executeSql(srv, "INSERT INTO T1 VALUES(?, ?, ?)", i, "Name-" + i, "Address-" + i);
+
+ checkDump(srv, DMP_NAME + 2, false);
+ }
+
+ /** */
+ private void checkDump(IgniteEx srv, String name, boolean first) throws Exception {
+ srv.snapshot().createDump(name).get(10_000L);
+
+ AtomicInteger cnt = new AtomicInteger();
+
+ TestDumpConsumer cnsmr = new TestDumpConsumer() {
+ @Override public void onTypes(Iterator types) {
+ super.onTypes(types);
+
+ assertTrue(types.hasNext());
+
+ BinaryType type = types.next();
+
+ assertFalse(types.hasNext());
+
+ assertTrue(type.typeName().startsWith("SQL_PUBLIC_T1"));
+ }
+
+ @Override public void onCacheConfigs(Iterator caches) {
+ super.onCacheConfigs(caches);
+
+ assertTrue(caches.hasNext());
+
+ StoredCacheData data = caches.next();
+
+ assertFalse(caches.hasNext());
+
+ assertTrue(data.sql());
+
+ CacheConfiguration ccfg = data.config();
+
+ assertEquals("T1", ccfg.getName());
+
+ Collection qes = data.queryEntities();
+
+ assertNotNull(qes);
+ assertEquals(1, qes.size());
+
+ QueryEntity qe = qes.iterator().next();
+
+ assertNotNull(qe);
+ assertEquals("T1", qe.getTableName());
+ assertEquals(first ? 2 : 3, qe.getFields().size());
+ assertTrue(qe.getFields().containsKey("ID"));
+ assertTrue(qe.getFields().containsKey("NAME"));
+ if (!first)
+ assertTrue(qe.getFields().containsKey("ADDRESS"));
+ }
+
+ @Override public void onPartition(int grp, int part, Iterator data) {
+ while (data.hasNext()) {
+ DumpEntry e = data.next();
+
+ assertNotNull(e);
+
+ BinaryObject val = (BinaryObject)e.value();
+
+ assertNotNull(val);
+ assertEquals("Name-" + e.key(), val.field("NAME"));
+
+ cnt.incrementAndGet();
+ }
+ }
+ };
+
+ new DumpReader(
+ new DumpReaderConfiguration(
+ new File(U.resolveWorkDirectory(U.defaultWorkDirectory(), srv.configuration().getSnapshotPath(), false), name),
+ cnsmr
+ ),
+ log
+ ).run();
+
+ assertEquals(first ? KEYS_CNT : (KEYS_CNT * 2), cnt.get());
+
+ cnsmr.check();
+ }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
index 219ecc25cf3a6..883274ad90488 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
@@ -22,6 +22,7 @@
import org.apache.ignite.internal.cdc.CacheEventsCdcTest;
import org.apache.ignite.internal.cdc.CdcIndexRebuildTest;
import org.apache.ignite.internal.cdc.SqlCdcTest;
+import org.apache.ignite.internal.dump.DumpCacheConfigTest;
import org.apache.ignite.internal.metric.SystemViewSelfTest;
import org.apache.ignite.internal.processors.cache.BigEntryQueryTest;
import org.apache.ignite.internal.processors.cache.BinaryMetadataConcurrentUpdateWithIndexesTest;
@@ -370,7 +371,9 @@
// CDC tests.
SqlCdcTest.class,
CacheEventsCdcTest.class,
- CdcIndexRebuildTest.class
+ CdcIndexRebuildTest.class,
+
+ DumpCacheConfigTest.class
})
public class IgniteBinaryCacheQueryTestSuite3 {
/** Setup lazy mode default. */