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: + *

    + * + */ +@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. */