You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/12/12 18:09:27 UTC

[GitHub] [ignite] ivandasch opened a new pull request, #10430: IGNITE-18198 WIP.

ivandasch opened a new pull request, #10430:
URL: https://github.com/apache/ignite/pull/10430

   Thank you for submitting the pull request to the Apache Ignite.
   
   In order to streamline the review of the contribution 
   we ask you to ensure the following steps have been taken:
   
   ### The Contribution Checklist
   - [ ] There is a single JIRA ticket related to the pull request. 
   - [ ] The web-link to the pull request is attached to the JIRA ticket.
   - [ ] The JIRA ticket has the _Patch Available_ state.
   - [ ] The pull request body describes changes that have been made. 
   The description explains _WHAT_ and _WHY_ was made instead of _HOW_.
   - [ ] The pull request title is treated as the final commit message. 
   The following pattern must be used: `IGNITE-XXXX Change summary` where `XXXX` - number of JIRA issue.
   - [ ] A reviewer has been mentioned through the JIRA comments 
   (see [the Maintainers list](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute#HowtoContribute-ReviewProcessandMaintainers)) 
   - [ ] The pull request has been checked by the Teamcity Bot and 
   the `green visa` attached to the JIRA ticket (see [TC.Bot: Check PR](https://mtcga.gridgain.com/prs.html))
   
   ### Notes
   - [How to Contribute](https://cwiki.apache.org/confluence/display/IGNITE/How+to+Contribute)
   - [Coding abbreviation rules](https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules)
   - [Coding Guidelines](https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines)
   - [Apache Ignite Teamcity Bot](https://cwiki.apache.org/confluence/display/IGNITE/Apache+Ignite+Teamcity+Bot)
   
   If you need any help, please email dev@ignite.apache.org or ask anу advice on http://asf.slack.com _#ignite_ channel.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054327820


##########
modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY;
+import static org.apache.ignite.events.EventType.EVTS_CLUSTER_SNAPSHOT;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED;
+
+/** */
+public class SnapshotCompressionBasicTest extends AbstractSnapshotSelfTest {
+    /** */
+    protected static final DiskPageCompression DISK_PAGE_COMPRESSION = DiskPageCompression.SNAPPY;
+
+    /** */
+    protected static final int PAGE_SIZE = 8 * 1024;
+
+    /** */
+    protected static final String SNAPSHOT_WITHOUT_HOLES = "testSnapshotWithoutHoles";
+
+    /** */
+    protected static final String SNAPSHOT_WITH_HOLES = "testSnapshotWithHoles";
+
+    /** */
+    protected static final long TIMEOUT = 30_000;
+
+    /** */
+    private static final Map<String, String> CACHES = new HashMap<>();
+
+    static {
+        CACHES.put("cache1", "group1");
+        CACHES.put("cache2", "group1");
+        CACHES.put("cache3", null);
+        CACHES.put("cache4", null);
+    }
+
+    /** */
+    private static final Set<String> COMPRESSED_CACHES = new HashSet<>();
+
+    static {
+        COMPRESSED_CACHES.add("cache1");
+        COMPRESSED_CACHES.add("cache2");
+        COMPRESSED_CACHES.add("cache3");
+    }
+
+    /** */
+    @Parameterized.Parameters(name = "Encryption={0}")
+    public static Collection<Boolean> encryptionParams() {
+        return Collections.singletonList(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration config = super.getConfiguration(igniteInstanceName);
+
+        config.getDataStorageConfiguration().setPageSize(PAGE_SIZE);
+
+        return config;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        cleanPersistenceDir();
+        createTestSnapshot();
+    }
+
+    /** {@inheritDoc} */
+    @Before
+    @Override public void beforeTestSnapshot() {
+        locEvts.clear();
+    }
+
+    /** {@inheritDoc} */
+    @After
+    @Override public void afterTestSnapshot() throws Exception {
+        if (G.allGrids().isEmpty())
+            return;
+
+        IgniteEx ig = grid(0);
+        for (String cacheName : ig.cacheNames()) {
+            IgniteCache cache = ig.cache(cacheName);
+
+            cache.destroy();
+        }
+
+        stopAllGrids();
+    }
+
+    /** */
+    @Test
+    public void testRestoreFullSnapshot() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                for (String cacheName : CACHES.keySet()) {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                }
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** */
+    @Test
+    public void testRestoreFail_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(this::failCompressionProcessor);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            GridTestUtils.assertThrows(log, () -> ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT),
+                IgniteException.class, "Snapshot contains compressed cache groups");
+        }
+    }
+
+
+    /** */
+    @Test
+    public void testRestoreNotCompressed_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i));
+
+        Collection<String> groupsWithoutCompression = CACHES.entrySet().stream()
+            .filter(e -> !COMPRESSED_CACHES.contains(e.getKey()))
+            .map(e -> e.getValue() != null ? e.getValue() : e.getKey())
+            .distinct().collect(Collectors.toList());
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, groupsWithoutCompression).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                CACHES.keySet().stream().filter(c -> !COMPRESSED_CACHES.contains(c)).forEach(cacheName -> {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                });
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** */
+    protected void createTestSnapshot() throws Exception {
+        CacheConfiguration[] caches = CACHES.entrySet().stream()
+            .map(cache -> {
+                CacheConfiguration config = new CacheConfiguration(cache.getKey());
+
+                if (cache.getValue() != null)
+                    config.setGroupName(cache.getValue());
+
+                if (COMPRESSED_CACHES.contains(cache.getKey()))
+                    config.setDiskPageCompression(DISK_PAGE_COMPRESSION);
+                else
+                    config.setDiskPageCompression(DiskPageCompression.DISABLED);
+
+                return config;
+            }).toArray(CacheConfiguration[]::new);
+
+        IgniteEx ignite = startGridsWithCache(3, 1000, valueBuilder(), caches);
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i, SNAPSHOT_WITHOUT_HOLES));
+
+        for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            ignite.snapshot().createSnapshot(snpName).get(TIMEOUT);
+
+            IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null)
+                .get().idleVerifyResult();
+
+            StringBuilder b = new StringBuilder();
+            res.print(b::append, true);
+
+            assertTrue("Exceptions: " + b, F.isEmpty(res.exceptions()));
+            assertTrue(F.isEmpty(res.exceptions()));
+        }
+
+        long withHolesSize = directorySize(
+            Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false).toString(), SNAPSHOT_WITH_HOLES));
+
+        long withoutHolesSize = directorySize(
+            Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false).toString(), SNAPSHOT_WITHOUT_HOLES));
+
+        assertTrue("withHolesSize < withoutHolesSize: " + withHolesSize + " <" + withoutHolesSize, withHolesSize < withoutHolesSize);
+
+        ignite.cacheNames().forEach(c -> ignite.getOrCreateCache(c).destroy());
+
+        G.stopAll(true);
+    }
+
+    /** */
+    private void failCompressionProcessor(Ignite ignite, String... snpNames) {
+        CompressionProcessor cmp = ((IgniteEx)ignite).context().compress();
+
+        CompressionProcessor spyCmp = Mockito.spy(cmp);
+
+        if (F.isEmpty(snpNames)) {
+            try {
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported();
+
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+        else {
+            for (String snpName : snpNames) {
+                try {
+                    Mockito.doAnswer(inv -> {
+                        if (snpName != null && ((Path)inv.getArgument(0)).endsWith(snpName))
+                            throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                        return null;
+                    }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        }
+
+        try {
+            Field cmpField = U.findField(GridKernalContextImpl.class, "compressProc");
+            cmpField.set(((IgniteEx)ignite).context(), spyCmp);
+        }
+        catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /** */
+    private static long directorySize(Path path) throws IOException {

Review Comment:
   FileUtils.sizeOfDirectory ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054293018


##########
modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY;
+import static org.apache.ignite.events.EventType.EVTS_CLUSTER_SNAPSHOT;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED;
+
+/** */
+public class SnapshotCompressionBasicTest extends AbstractSnapshotSelfTest {
+    /** */
+    protected static final DiskPageCompression DISK_PAGE_COMPRESSION = DiskPageCompression.SNAPPY;
+
+    /** */
+    protected static final int PAGE_SIZE = 8 * 1024;
+
+    /** */
+    protected static final String SNAPSHOT_WITHOUT_HOLES = "testSnapshotWithoutHoles";
+
+    /** */
+    protected static final String SNAPSHOT_WITH_HOLES = "testSnapshotWithHoles";
+
+    /** */
+    protected static final long TIMEOUT = 30_000;

Review Comment:
   Can we reuse getTestTimeout or AbstractSnapshotSelfTest#TIMEOUT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] alex-plekhanov commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1056080925


##########
modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java:
##########
@@ -0,0 +1,392 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+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.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.persistence.file.FileIO;
+import org.apache.ignite.internal.processors.cache.persistence.file.RandomAccessFileIO;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY;
+import static org.apache.ignite.events.EventType.EVTS_CLUSTER_SNAPSHOT;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED;
+
+/** */
+public class SnapshotCompressionBasicTest extends AbstractSnapshotSelfTest {
+    /** */
+    protected static final DiskPageCompression DISK_PAGE_COMPRESSION = DiskPageCompression.SNAPPY;
+
+    /** */
+    protected static final int PAGE_SIZE = 8 * 1024;
+
+    /** */
+    protected static final String SNAPSHOT_WITHOUT_HOLES = "testSnapshotWithoutHoles";
+
+    /** */
+    protected static final String SNAPSHOT_WITH_HOLES = "testSnapshotWithHoles";
+
+    /** */
+    protected static final long TIMEOUT = 30_000;
+
+    /** */
+    private static final Map<String, String> CACHES = new HashMap<>();
+
+    static {
+        CACHES.put("cache1", "group1");
+        CACHES.put("cache2", "group1");
+        CACHES.put("cache3", null);
+        CACHES.put("cache4", null);
+    }
+
+    /** */
+    private static final Set<String> COMPRESSED_CACHES = new HashSet<>();
+
+    static {
+        COMPRESSED_CACHES.add("cache1");
+        COMPRESSED_CACHES.add("cache2");
+        COMPRESSED_CACHES.add("cache3");
+    }
+
+    /** */
+    @Parameterized.Parameters(name = "Encryption={0}")
+    public static Collection<Boolean> encryptionParams() {
+        return Collections.singletonList(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration config = super.getConfiguration(igniteInstanceName);
+
+        config.getDataStorageConfiguration().setPageSize(PAGE_SIZE);
+
+        return config;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        cleanPersistenceDir();
+        createTestSnapshot();
+    }
+
+    /** {@inheritDoc} */
+    @Before
+    @Override public void beforeTestSnapshot() {
+        locEvts.clear();
+    }
+
+    /** {@inheritDoc} */
+    @After
+    @Override public void afterTestSnapshot() throws Exception {
+        if (G.allGrids().isEmpty())
+            return;
+
+        IgniteEx ig = grid(0);
+        for (String cacheName : ig.cacheNames()) {
+            IgniteCache cache = ig.cache(cacheName);
+
+            cache.destroy();
+        }
+
+        stopAllGrids();
+    }
+
+    /** */
+    @Test
+    public void testRestoreFullSnapshot() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                for (String cacheName : CACHES.keySet()) {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                }
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** */
+    @Test
+    public void testRestoreFail_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(this::failCompressionProcessor);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            GridTestUtils.assertThrows(log, () -> ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT),
+                IgniteException.class, "Snapshot contains compressed cache groups");
+        }
+    }
+
+
+    /** */
+    @Test
+    public void testRestoreNotCompressed_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i));
+
+        Collection<String> groupsWithoutCompression = CACHES.entrySet().stream()
+            .filter(e -> !COMPRESSED_CACHES.contains(e.getKey()))
+            .map(e -> e.getValue() != null ? e.getValue() : e.getKey())
+            .distinct().collect(Collectors.toList());
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, groupsWithoutCompression).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                CACHES.keySet().stream().filter(c -> !COMPRESSED_CACHES.contains(c)).forEach(cacheName -> {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                });
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Function<Integer, Object> valueBuilder() {
+        return i -> new Value("name_" + i);
+    }
+
+    /** */
+    protected void createTestSnapshot() throws Exception {
+        CacheConfiguration[] caches = CACHES.entrySet().stream()
+            .map(cache -> {
+                CacheConfiguration config = new CacheConfiguration(cache.getKey());
+
+                config.setQueryEntities(Collections.singletonList(
+                    new QueryEntity()
+                        .setKeyType(Integer.class.getName())
+                        .setValueType(Value.class.getName())
+                        .addQueryField("id", Integer.class.getName(), null)
+                        .addQueryField("name", String.class.getName(), null)
+                        .setIndexes(F.asList(new QueryIndex("name")))
+                ));
+
+                if (cache.getValue() != null)
+                    config.setGroupName(cache.getValue());
+
+                if (COMPRESSED_CACHES.contains(cache.getKey()))
+                    config.setDiskPageCompression(DISK_PAGE_COMPRESSION);
+                else
+                    config.setDiskPageCompression(DiskPageCompression.DISABLED);
+
+                return config;
+            }).toArray(CacheConfiguration[]::new);
+
+        IgniteEx ignite = startGridsWithCache(3, 1000, valueBuilder(), caches);
+
+        forceCheckpoint();
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i, SNAPSHOT_WITHOUT_HOLES));
+
+        for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            ignite.snapshot().createSnapshot(snpName).get(TIMEOUT);
+
+            IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null)
+                .get().idleVerifyResult();
+
+            StringBuilder b = new StringBuilder();
+            res.print(b::append, true);
+
+            assertTrue("Exceptions: " + b, F.isEmpty(res.exceptions()));
+            assertTrue(F.isEmpty(res.exceptions()));
+        }
+
+        Path withHolesPath = Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false)
+            .toString(), SNAPSHOT_WITH_HOLES);
+
+        Path withoutHolesPath = Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false)
+            .toString(), SNAPSHOT_WITHOUT_HOLES);
+
+        long withHolesSize = directorySize(withHolesPath);
+        long withoutHolesSize = directorySize(withoutHolesPath);
+
+        assertTrue("withHolesSize < withoutHolesSize: " + withHolesSize + " < " + withoutHolesSize,
+            withHolesSize < withoutHolesSize);
+
+        long idxWithHolesSize = directorySize(withHolesPath, "index\\.bin");
+        long idxWithoutHolesSize = directorySize(withoutHolesPath, "index\\.bin");
+
+        assertTrue("idxWithHolesSize < idxWithoutHolesSize: " + idxWithHolesSize + " < " + idxWithoutHolesSize,
+            idxWithHolesSize < idxWithoutHolesSize);
+
+        ignite.cacheNames().forEach(c -> ignite.getOrCreateCache(c).destroy());
+
+        G.stopAll(true);
+    }
+
+    /** */
+    private void failCompressionProcessor(Ignite ignite, String... snpNames) {
+        CompressionProcessor cmp = ((IgniteEx)ignite).context().compress();
+
+        CompressionProcessor spyCmp = Mockito.spy(cmp);
+
+        if (F.isEmpty(snpNames)) {
+            try {
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported();
+
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+        else {
+            for (String snpName : snpNames) {
+                try {
+                    Mockito.doAnswer(inv -> {
+                        if (snpName != null && ((Path)inv.getArgument(0)).endsWith(snpName))
+                            throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                        return null;
+                    }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        }
+
+        try {
+            Field cmpField = U.findField(GridKernalContextImpl.class, "compressProc");
+            cmpField.set(((IgniteEx)ignite).context(), spyCmp);

Review Comment:
   `((GridKernalContextImpl)((IgniteEx)ignite).context()).add(spyCmp);`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054305843


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -795,6 +797,10 @@ private IgniteInternalFuture<SnapshotOperationResponse> initLocalSnapshotStartSt
         }
 
         List<Integer> grpIds = new ArrayList<>(F.viewReadOnly(req.groups(), CU::cacheId));
+        List<Integer> comprGrpIds = grpIds.stream().filter(i -> {

Review Comment:
   Agree, there is no need to create new List



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054300577


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);
+        GridUnsafe.zeroMemory(GridUnsafe.bufferAddress(buffer), buffer.capacity());
+
+        try {
+            try (RandomAccessFileIO testFileIO = new RandomAccessFileIO(testFile.toFile(), CREATE, WRITE)) {
+                testFileIO.writeFully(buffer);
+
+                testFileIO.punchHole(fsBlockSz, fsBlockSz);
+            }
+            finally {
+                Files.deleteIfExists(testFile);
+            }
+        }
+        catch (Exception e) {

Review Comment:
   ```
   /** */
       static {
           NativeFileSystem x = null;
   
           try {
               if (IgniteComponentType.COMPRESSION.inClassPath()) {
                   if (U.isLinux())
                       x = U.newInstance(NATIVE_FS_LINUX_CLASS);
               }
           }
           catch (Throwable e) {
               err = e;
           }
   
           fs = x;
       }
   ```
   in `FileSystemUtils`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054284410


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);
+        GridUnsafe.zeroMemory(GridUnsafe.bufferAddress(buffer), buffer.capacity());
+
+        try {
+            try (RandomAccessFileIO testFileIO = new RandomAccessFileIO(testFile.toFile(), CREATE, WRITE)) {
+                testFileIO.writeFully(buffer);
+
+                testFileIO.punchHole(fsBlockSz, fsBlockSz);
+            }
+            finally {
+                Files.deleteIfExists(testFile);
+            }
+        }
+        catch (Exception e) {

Review Comment:
   I think `LinkageError` should be caught too.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054314341


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java:
##########
@@ -174,6 +188,16 @@ public Map<Integer, Set<Integer>> partitions() {
         return Collections.unmodifiableMap(locParts);
     }
 
+    /** */
+    public boolean isGroupWithCompresion(int grpId) {
+        return hasComprGrps && comprGrpIds.contains(grpId);
+    }
+
+    /** */
+    public boolean hasCompressedGroups() {
+        return hasComprGrps;

Review Comment:
   Can we use F.isEmpty instead of a flag?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054319776


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotDeltaTest.java:
##########
@@ -77,7 +80,8 @@ public static Collection<Object[]> parameters() {
     @Test
     public void testSendDelta() throws Exception {
         int keys = 10_000;
-        byte[] payload = new byte[DFLT_PAGE_SIZE / 2];
+        int pageSz = IgniteSystemProperties.getInteger(IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, DFLT_PAGE_SIZE);

Review Comment:
   AbstractSnapshotSelfTest#PAGE_SIZE ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054333118


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");

Review Comment:
   `public static File createTempFile(String prefix, String suffix, File directory)` can be used



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] alex-plekhanov commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
alex-plekhanov commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1055467359


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java:
##########
@@ -144,8 +144,29 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     /** Marshaller context implementation. */
     private MarshallerContextImpl marshallerCtx;
 
+    /** */
+    @Nullable private CompressionProcessor cmpProc;

Review Comment:
   Abbriviation `cmp` is usually used for `compare`, let's use `compressProc` here (in other files too)



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java:
##########
@@ -174,13 +181,32 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
                                  snpEncrKeyProvider : null).createPageStore(getTypeByPartId(partId), part::toPath, val -> {})
                     ) {
                         if (partId == INDEX_PARTITION) {
-                            checkPartitionsPageCrcSum(() -> pageStore, INDEX_PARTITION, FLAG_IDX);
+                            if (punchHoleEnabled && meta.isGroupWithCompresion(grpId) && type() == SnapshotHandlerType.CREATE) {
+                                checkPartitionsPageCrcSum(() -> pageStore, INDEX_PARTITION, FLAG_IDX, (id, buffer) -> {
+                                    if (PageIO.getCompressionType(buffer) == CompressionProcessor.UNCOMPRESSED_PAGE)
+                                        return;
+
+                                    int comprPageSz = PageIO.getCompressedSize(buffer);
+
+                                    if (comprPageSz < pageStore.getPageSize()) {
+                                        try {
+                                            pageStore.punchHole(id, comprPageSz);
+                                        }
+                                        catch (Exception ignored) {
+                                            // No-op
+                                        }
+                                    }
+                                });
+                            }
+                            else if (!skipHash())
+                                checkPartitionsPageCrcSum(() -> pageStore, INDEX_PARTITION, FLAG_IDX);
 
                             return null;
                         }
 
                         if (grpId == MetaStorage.METASTORAGE_CACHE_ID) {
-                            checkPartitionsPageCrcSum(() -> pageStore, partId, FLAG_DATA);
+                            if (!skipHash())

Review Comment:
   Why this check was removed for "skipHash"? As far as I understand "skipHash" mean values check should be skipped, but not physical page checks.



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java:
##########
@@ -89,6 +109,9 @@ public static void checkPartitionsPageCrcSum(
                 buf.clear();
 
                 pageStore.read(pageId, buf, true, true);
+
+                if (pagePostProcessor != null)
+                    pagePostProcessor.accept(partId, buf);

Review Comment:
   `partId` -> `pageId`



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java:
##########
@@ -191,6 +217,22 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
 
                         long pageAddr = GridUnsafe.bufferAddress(pageBuff);
 
+                        if (PageIO.getCompressionType(pageBuff) != CompressionProcessor.UNCOMPRESSED_PAGE) {
+                            int comprPageSz = PageIO.getCompressedSize(pageBuff);
+                            long pageId = PageIO.getPageId(pageAddr);
+
+                            snpCtx.compress().decompressPage(pageBuff, pageStore.getPageSize());
+
+                            if (punchHoleEnabled && comprPageSz < pageStore.getPageSize()) {
+                                try {
+                                    pageStore.punchHole(pageId, comprPageSz);

Review Comment:
   Shouldn't we punch the hole only on snapshot create operation?



##########
modules/compress/src/test/java/org/apache/ignite/testsuites/IgnitePdsCompressionTestSuite.java:
##########
@@ -60,7 +64,15 @@ public static List<Class<?>> suite() {
         suite.add(IgnitePdsCheckpointSimulationWithRealCpDisabledAndWalCompressionTest.class);
         suite.add(WalCompactionAndPageCompressionTest.class);
 
+        // Snapshots

Review Comment:
   Point at the end of line



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java:
##########
@@ -213,14 +255,21 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
                         // There is no `primary` partitions for snapshot.
                         PartitionKeyV2 key = new PartitionKeyV2(grpId, partId, grpName);
 
+                        GridIterator<CacheDataRow> partRowIter;
+                        if (punchHoleEnabled && meta.isGroupWithCompresion(grpId) && type() == SnapshotHandlerType.CREATE)
+                            partRowIter = snpMgr.partitionRowIterator(snpCtx, grpName, partId, pageStore, true);

Review Comment:
   This makes values hash calculation unskippable. If we don't want hash calculation we can use lightweight check `checkPartitionsPageCrcSum`. I propose to change logic in this method a little bit:
   - undo changes in partitionRowIterator, this iterator should only be called on restore operation (skipHash == false)
   - add `checkPartitionsPageCrcSum` with hole punching for create operation (at least when group is compessed and punch holes is enabled, but perhaps always, I don't know why we do CRC check for index partition and skip this check for regular partition)
   - undo dedicated hole punching for metadata page 



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java:
##########
@@ -943,12 +945,28 @@ protected void init() throws IOException {
                     if (!store.read(pageId, locBuf, true))
                         return;
 
-                    locBuf.flip();
+                    locBuf.limit(locBuf.capacity());
+                    locBuf.position(0);
 
                     writePage0(pageId, locBuf);
                 }
                 else {
                     // Direct buffer is needs to be written, associated checkpoint not finished yet.
+                    if (PageIO.getCompressionType(GridUnsafe.bufferAddress(buf)) != CompressionProcessor.UNCOMPRESSED_PAGE) {
+                        final ByteBuffer locBuf = locBuff.get();
+
+                        assert locBuf.capacity() == store.getPageSize();
+
+                        locBuf.clear();
+
+                        GridUnsafe.copyOffheapOffheap(GridUnsafe.bufferAddress(buf), GridUnsafe.bufferAddress(locBuf), buf.limit());
+
+                        locBuf.limit(locBuf.capacity());
+                        locBuf.position(0);
+
+                        buf = locBuf;
+                    }

Review Comment:
   Why can't we just set and restore `limit` for `buf`?



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java:
##########
@@ -943,12 +945,28 @@ protected void init() throws IOException {
                     if (!store.read(pageId, locBuf, true))
                         return;
 
-                    locBuf.flip();
+                    locBuf.limit(locBuf.capacity());
+                    locBuf.position(0);

Review Comment:
   `locBuf.clear()`?



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -1458,6 +1465,38 @@ public IgniteInternalFuture<SnapshotPartitionsVerifyTaskResult> checkSnapshot(
                             return;
                         }
 
+                        if (meta.hasCompressedGroups() && grpIds.keySet().stream().anyMatch(meta::isGroupWithCompresion)) {
+                            try {
+                                File dbPath = kctx0.pdsFolderResolver().resolveFolders().persistentStoreRootPath();
+                                int pageSize = kctx0.config().getDataStorageConfiguration().getPageSize();
+
+                                kctx0.compress().checkPageCompressionSupported(dbPath.toPath(), pageSize);

Review Comment:
   This method also checks that FS is hole-punchable, but to check snapshot we can proceed even on FS without this ability, so we can just check compression processor is enabled (method `checkPageCompressionSupported()` without parameters) 



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -1458,6 +1465,38 @@ public IgniteInternalFuture<SnapshotPartitionsVerifyTaskResult> checkSnapshot(
                             return;
                         }
 
+                        if (meta.hasCompressedGroups() && grpIds.keySet().stream().anyMatch(meta::isGroupWithCompresion)) {
+                            try {
+                                File dbPath = kctx0.pdsFolderResolver().resolveFolders().persistentStoreRootPath();
+                                int pageSize = kctx0.config().getDataStorageConfiguration().getPageSize();
+
+                                kctx0.compress().checkPageCompressionSupported(dbPath.toPath(), pageSize);
+                            }
+                            catch (IgniteCheckedException e) {
+                                String msg;
+                                if (grpIds.isEmpty()) {
+                                    msg = "Snapshot '" + meta.snapshotName() + "' contains compressed cache groups while " +

Review Comment:
   Unreachable code



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java:
##########
@@ -688,6 +689,28 @@ private SnapshotRestoreContext prepareContext(
                 if (!F.isEmpty(req.groups()) && !req.groups().contains(grpName))
                     continue;
 
+                if (!skipCompressCheck && meta.isGroupWithCompresion(CU.cacheId(grpName))) {
+                    try {
+                        File path = ctx.pdsFolderResolver().resolveFolders().persistentStoreRootPath();
+
+                        ctx.compress().checkPageCompressionSupported(path.toPath(), meta.pageSize());

Review Comment:
   This method also checks that FS is hole-punchable, but to restore we can proceed even on FS without this ability since in any way we don't punch the holes during restore. So we can just check compression processor is enabled (method `checkPageCompressionSupported()` without parameters).



##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotPartitionsVerifyHandler.java:
##########
@@ -174,13 +181,32 @@ public SnapshotPartitionsVerifyHandler(GridCacheSharedContext<?, ?> cctx) {
                                  snpEncrKeyProvider : null).createPageStore(getTypeByPartId(partId), part::toPath, val -> {})
                     ) {
                         if (partId == INDEX_PARTITION) {
-                            checkPartitionsPageCrcSum(() -> pageStore, INDEX_PARTITION, FLAG_IDX);
+                            if (punchHoleEnabled && meta.isGroupWithCompresion(grpId) && type() == SnapshotHandlerType.CREATE) {
+                                checkPartitionsPageCrcSum(() -> pageStore, INDEX_PARTITION, FLAG_IDX, (id, buffer) -> {
+                                    if (PageIO.getCompressionType(buffer) == CompressionProcessor.UNCOMPRESSED_PAGE)
+                                        return;
+
+                                    int comprPageSz = PageIO.getCompressedSize(buffer);
+
+                                    if (comprPageSz < pageStore.getPageSize()) {
+                                        try {
+                                            pageStore.punchHole(id, comprPageSz);
+                                        }
+                                        catch (Exception ignored) {
+                                            // No-op
+                                        }
+                                    }
+                                });
+                            }
+                            else if (!skipHash())

Review Comment:
   Why this check was removed for "skipHash"? As far as I understand "skipHash" mean values check should be skipped, but not physical page checks.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054298861


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);

Review Comment:
   Good catch, thank you



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054302430


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java:
##########
@@ -73,6 +74,24 @@ public static void checkPartitionsPageCrcSum(
         IgniteThrowableSupplier<FilePageStore> pageStoreSup,
         int partId,
         byte pageType
+    ) {
+        checkPartitionsPageCrcSum(pageStoreSup, partId, pageType, null);
+    }
+
+    /**
+     * Checks CRC sum of pages with {@code pageType} page type stored in partition with {@code partId} id
+     * and associated with cache group.
+     *
+     * @param pageStoreSup Page store supplier.
+     * @param partId Partition id.
+     * @param pageType Page type. Possible types {@link PageIdAllocator#FLAG_DATA}, {@link PageIdAllocator#FLAG_IDX}
+     *      and {@link PageIdAllocator#FLAG_AUX}.
+     */
+    public static void checkPartitionsPageCrcSum(
+        IgniteThrowableSupplier<FilePageStore> pageStoreSup,
+        int partId,
+        byte pageType,
+        @Nullable IgniteThrowableBiConsumer<Integer, ByteBuffer> pagePostProcessor

Review Comment:
   I don't want to limit user with necessity to catch all checked exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054328844


##########
modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY;
+import static org.apache.ignite.events.EventType.EVTS_CLUSTER_SNAPSHOT;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED;
+
+/** */
+public class SnapshotCompressionBasicTest extends AbstractSnapshotSelfTest {
+    /** */
+    protected static final DiskPageCompression DISK_PAGE_COMPRESSION = DiskPageCompression.SNAPPY;
+
+    /** */
+    protected static final int PAGE_SIZE = 8 * 1024;
+
+    /** */
+    protected static final String SNAPSHOT_WITHOUT_HOLES = "testSnapshotWithoutHoles";
+
+    /** */
+    protected static final String SNAPSHOT_WITH_HOLES = "testSnapshotWithHoles";
+
+    /** */
+    protected static final long TIMEOUT = 30_000;
+
+    /** */
+    private static final Map<String, String> CACHES = new HashMap<>();
+
+    static {
+        CACHES.put("cache1", "group1");
+        CACHES.put("cache2", "group1");
+        CACHES.put("cache3", null);
+        CACHES.put("cache4", null);
+    }
+
+    /** */
+    private static final Set<String> COMPRESSED_CACHES = new HashSet<>();
+
+    static {
+        COMPRESSED_CACHES.add("cache1");
+        COMPRESSED_CACHES.add("cache2");
+        COMPRESSED_CACHES.add("cache3");
+    }
+
+    /** */
+    @Parameterized.Parameters(name = "Encryption={0}")
+    public static Collection<Boolean> encryptionParams() {
+        return Collections.singletonList(false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration config = super.getConfiguration(igniteInstanceName);
+
+        config.getDataStorageConfiguration().setPageSize(PAGE_SIZE);
+
+        return config;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        cleanPersistenceDir();
+        createTestSnapshot();
+    }
+
+    /** {@inheritDoc} */
+    @Before
+    @Override public void beforeTestSnapshot() {
+        locEvts.clear();
+    }
+
+    /** {@inheritDoc} */
+    @After
+    @Override public void afterTestSnapshot() throws Exception {
+        if (G.allGrids().isEmpty())
+            return;
+
+        IgniteEx ig = grid(0);
+        for (String cacheName : ig.cacheNames()) {
+            IgniteCache cache = ig.cache(cacheName);
+
+            cache.destroy();
+        }
+
+        stopAllGrids();
+    }
+
+    /** */
+    @Test
+    public void testRestoreFullSnapshot() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                for (String cacheName : CACHES.keySet()) {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                }
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** */
+    @Test
+    public void testRestoreFail_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(this::failCompressionProcessor);
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            GridTestUtils.assertThrows(log, () -> ignite.snapshot().restoreSnapshot(snpName, null).get(TIMEOUT),
+                IgniteException.class, "Snapshot contains compressed cache groups");
+        }
+    }
+
+
+    /** */
+    @Test
+    public void testRestoreNotCompressed_OnGridWithoutCompression() throws Exception {
+        IgniteEx ignite = startGrids(3);
+        ignite.events().localListen(e -> locEvts.add(e.type()), EVTS_CLUSTER_SNAPSHOT);
+        ignite.cluster().state(ClusterState.ACTIVE);
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i));
+
+        Collection<String> groupsWithoutCompression = CACHES.entrySet().stream()
+            .filter(e -> !COMPRESSED_CACHES.contains(e.getKey()))
+            .map(e -> e.getValue() != null ? e.getValue() : e.getKey())
+            .distinct().collect(Collectors.toList());
+
+        for (String snpName: Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            try {
+                ignite.snapshot().restoreSnapshot(snpName, groupsWithoutCompression).get(TIMEOUT);
+
+                waitForEvents(EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED, EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED);
+
+                CACHES.keySet().stream().filter(c -> !COMPRESSED_CACHES.contains(c)).forEach(cacheName -> {
+                    IgniteCache cache = ignite.cache(cacheName);
+
+                    assertCacheKeys(cache, 1000);
+
+                    cache.destroy();
+                });
+            }
+            finally {
+                locEvts.clear();
+            }
+        }
+    }
+
+    /** */
+    protected void createTestSnapshot() throws Exception {
+        CacheConfiguration[] caches = CACHES.entrySet().stream()
+            .map(cache -> {
+                CacheConfiguration config = new CacheConfiguration(cache.getKey());
+
+                if (cache.getValue() != null)
+                    config.setGroupName(cache.getValue());
+
+                if (COMPRESSED_CACHES.contains(cache.getKey()))
+                    config.setDiskPageCompression(DISK_PAGE_COMPRESSION);
+                else
+                    config.setDiskPageCompression(DiskPageCompression.DISABLED);
+
+                return config;
+            }).toArray(CacheConfiguration[]::new);
+
+        IgniteEx ignite = startGridsWithCache(3, 1000, valueBuilder(), caches);
+
+        G.allGrids().forEach(i -> failCompressionProcessor(i, SNAPSHOT_WITHOUT_HOLES));
+
+        for (String snpName : Arrays.asList(SNAPSHOT_WITH_HOLES, SNAPSHOT_WITHOUT_HOLES)) {
+            ignite.snapshot().createSnapshot(snpName).get(TIMEOUT);
+
+            IdleVerifyResultV2 res = ignite.context().cache().context().snapshotMgr().checkSnapshot(snpName, null)
+                .get().idleVerifyResult();
+
+            StringBuilder b = new StringBuilder();
+            res.print(b::append, true);
+
+            assertTrue("Exceptions: " + b, F.isEmpty(res.exceptions()));
+            assertTrue(F.isEmpty(res.exceptions()));
+        }
+
+        long withHolesSize = directorySize(
+            Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false).toString(), SNAPSHOT_WITH_HOLES));
+
+        long withoutHolesSize = directorySize(
+            Paths.get(U.resolveWorkDirectory(U.defaultWorkDirectory(), DFLT_SNAPSHOT_DIRECTORY, false).toString(), SNAPSHOT_WITHOUT_HOLES));
+
+        assertTrue("withHolesSize < withoutHolesSize: " + withHolesSize + " <" + withoutHolesSize, withHolesSize < withoutHolesSize);
+
+        ignite.cacheNames().forEach(c -> ignite.getOrCreateCache(c).destroy());
+
+        G.stopAll(true);
+    }
+
+    /** */
+    private void failCompressionProcessor(Ignite ignite, String... snpNames) {
+        CompressionProcessor cmp = ((IgniteEx)ignite).context().compress();
+
+        CompressionProcessor spyCmp = Mockito.spy(cmp);
+
+        if (F.isEmpty(snpNames)) {
+            try {
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported();
+
+                Mockito.doAnswer(inv -> {
+                    throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+        else {
+            for (String snpName : snpNames) {
+                try {
+                    Mockito.doAnswer(inv -> {
+                        if (snpName != null && ((Path)inv.getArgument(0)).endsWith(snpName))
+                            throw new IgniteCheckedException(new IgniteException("errno: -12"));
+                        return null;
+                    }).when(spyCmp).checkPageCompressionSupported(Mockito.any(), Mockito.anyInt());
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
+                }
+            }
+        }
+
+        try {
+            Field cmpField = U.findField(GridKernalContextImpl.class, "compressProc");
+            cmpField.set(((IgniteEx)ignite).context(), spyCmp);
+        }
+        catch (IllegalAccessException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /** */
+    private static long directorySize(Path path) throws IOException {

Review Comment:
   What is the FileUtils? I don't know any



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1055781368


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotFutureTask.java:
##########
@@ -943,12 +945,28 @@ protected void init() throws IOException {
                     if (!store.read(pageId, locBuf, true))
                         return;
 
-                    locBuf.flip();
+                    locBuf.limit(locBuf.capacity());
+                    locBuf.position(0);
 
                     writePage0(pageId, locBuf);
                 }
                 else {
                     // Direct buffer is needs to be written, associated checkpoint not finished yet.
+                    if (PageIO.getCompressionType(GridUnsafe.bufferAddress(buf)) != CompressionProcessor.UNCOMPRESSED_PAGE) {
+                        final ByteBuffer locBuf = locBuff.get();
+
+                        assert locBuf.capacity() == store.getPageSize();
+
+                        locBuf.clear();
+
+                        GridUnsafe.copyOffheapOffheap(GridUnsafe.bufferAddress(buf), GridUnsafe.bufferAddress(locBuf), buf.limit());
+
+                        locBuf.limit(locBuf.capacity());
+                        locBuf.position(0);
+
+                        buf = locBuf;
+                    }

Review Comment:
   Sometimes the buf's length is less than the locBuf's length.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1055782080


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotRestoreProcess.java:
##########
@@ -688,6 +689,28 @@ private SnapshotRestoreContext prepareContext(
                 if (!F.isEmpty(req.groups()) && !req.groups().contains(grpName))
                     continue;
 
+                if (!skipCompressCheck && meta.isGroupWithCompresion(CU.cacheId(grpName))) {
+                    try {
+                        File path = ctx.pdsFolderResolver().resolveFolders().persistentStoreRootPath();
+
+                        ctx.compress().checkPageCompressionSupported(path.toPath(), meta.pageSize());

Review Comment:
   Yep, but it will fail later on working grid. It is not safe.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054301680


##########
modules/compress/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotCompressionBasicTest.java:
##########
@@ -0,0 +1,324 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.persistence.snapshot;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DiskPageCompression;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContextImpl;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.verify.IdleVerifyResultV2;
+import org.apache.ignite.internal.processors.compress.CompressionProcessor;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runners.Parameterized;
+import org.mockito.Mockito;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.DFLT_SNAPSHOT_DIRECTORY;
+import static org.apache.ignite.events.EventType.EVTS_CLUSTER_SNAPSHOT;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_FINISHED;
+import static org.apache.ignite.events.EventType.EVT_CLUSTER_SNAPSHOT_RESTORE_STARTED;
+
+/** */
+public class SnapshotCompressionBasicTest extends AbstractSnapshotSelfTest {
+    /** */
+    protected static final DiskPageCompression DISK_PAGE_COMPRESSION = DiskPageCompression.SNAPPY;
+
+    /** */
+    protected static final int PAGE_SIZE = 8 * 1024;
+
+    /** */
+    protected static final String SNAPSHOT_WITHOUT_HOLES = "testSnapshotWithoutHoles";
+
+    /** */
+    protected static final String SNAPSHOT_WITH_HOLES = "testSnapshotWithHoles";
+
+    /** */
+    protected static final long TIMEOUT = 30_000;

Review Comment:
   No, it is not enough



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054284410


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);
+        GridUnsafe.zeroMemory(GridUnsafe.bufferAddress(buffer), buffer.capacity());
+
+        try {
+            try (RandomAccessFileIO testFileIO = new RandomAccessFileIO(testFile.toFile(), CREATE, WRITE)) {
+                testFileIO.writeFully(buffer);
+
+                testFileIO.punchHole(fsBlockSz, fsBlockSz);
+            }
+            finally {
+                Files.deleteIfExists(testFile);
+            }
+        }
+        catch (Exception e) {

Review Comment:
   I think `LinkageError` should be caught too (for example may throw `UnsatisfiedLinkError`).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054291151


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");

Review Comment:
   Why not `File createTempFile`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054327547


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java:
##########
@@ -221,6 +253,15 @@ private void readObject(java.io.ObjectInputStream s) throws IOException, ClassNo
 
             locParts.put(grpId, parts);
         }
+
+        if (hasComprGrps) {
+            int sz = s.readInt();

Review Comment:
   Yep, it is probably better.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054302430


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java:
##########
@@ -73,6 +74,24 @@ public static void checkPartitionsPageCrcSum(
         IgniteThrowableSupplier<FilePageStore> pageStoreSup,
         int partId,
         byte pageType
+    ) {
+        checkPartitionsPageCrcSum(pageStoreSup, partId, pageType, null);
+    }
+
+    /**
+     * Checks CRC sum of pages with {@code pageType} page type stored in partition with {@code partId} id
+     * and associated with cache group.
+     *
+     * @param pageStoreSup Page store supplier.
+     * @param partId Partition id.
+     * @param pageType Page type. Possible types {@link PageIdAllocator#FLAG_DATA}, {@link PageIdAllocator#FLAG_IDX}
+     *      and {@link PageIdAllocator#FLAG_AUX}.
+     */
+    public static void checkPartitionsPageCrcSum(
+        IgniteThrowableSupplier<FilePageStore> pageStoreSup,
+        int partId,
+        byte pageType,
+        @Nullable IgniteThrowableBiConsumer<Integer, ByteBuffer> pagePostProcessor

Review Comment:
   I don't what to limit user with necessity to catch all checked exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054302894


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -795,6 +797,10 @@ private IgniteInternalFuture<SnapshotOperationResponse> initLocalSnapshotStartSt
         }
 
         List<Integer> grpIds = new ArrayList<>(F.viewReadOnly(req.groups(), CU::cacheId));
+        List<Integer> comprGrpIds = grpIds.stream().filter(i -> {

Review Comment:
   I suggest to use lightweight filter: `F.view(grpIds, id -> {..}`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054314631


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java:
##########
@@ -221,6 +253,15 @@ private void readObject(java.io.ObjectInputStream s) throws IOException, ClassNo
 
             locParts.put(grpId, parts);
         }
+
+        if (hasComprGrps) {
+            int sz = s.readInt();

Review Comment:
   U.readSet / U.writeCollection can be reused



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054323415


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java:
##########
@@ -174,6 +188,16 @@ public Map<Integer, Set<Integer>> partitions() {
         return Collections.unmodifiableMap(locParts);
     }
 
+    /** */
+    public boolean isGroupWithCompresion(int grpId) {
+        return hasComprGrps && comprGrpIds.contains(grpId);
+    }
+
+    /** */
+    public boolean hasCompressedGroups() {
+        return hasComprGrps;

Review Comment:
   Flag is better here, because of backward compatibility. We can add field and if field has not been serialized in older version, this flag will be false. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch merged pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch merged PR #10430:
URL: https://github.com/apache/ignite/pull/10430


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054297373


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");

Review Comment:
   Because it will be created in /tmp. We should check exactly the given path



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054287716


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);

Review Comment:
   Seems we need to call  GridUnsafe.freeBuffer



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054325045


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotDeltaTest.java:
##########
@@ -77,7 +80,8 @@ public static Collection<Object[]> parameters() {
     @Test
     public void testSendDelta() throws Exception {
         int keys = 10_000;
-        byte[] payload = new byte[DFLT_PAGE_SIZE / 2];
+        int pageSz = IgniteSystemProperties.getInteger(IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, DFLT_PAGE_SIZE);

Review Comment:
   Oh, I forgot  that i had rewritten it already :)



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054296485


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/verify/IdleVerifyUtility.java:
##########
@@ -73,6 +74,24 @@ public static void checkPartitionsPageCrcSum(
         IgniteThrowableSupplier<FilePageStore> pageStoreSup,
         int partId,
         byte pageType
+    ) {
+        checkPartitionsPageCrcSum(pageStoreSup, partId, pageType, null);
+    }
+
+    /**
+     * Checks CRC sum of pages with {@code pageType} page type stored in partition with {@code partId} id
+     * and associated with cache group.
+     *
+     * @param pageStoreSup Page store supplier.
+     * @param partId Partition id.
+     * @param pageType Page type. Possible types {@link PageIdAllocator#FLAG_DATA}, {@link PageIdAllocator#FLAG_IDX}
+     *      and {@link PageIdAllocator#FLAG_AUX}.
+     */
+    public static void checkPartitionsPageCrcSum(
+        IgniteThrowableSupplier<FilePageStore> pageStoreSup,
+        int partId,
+        byte pageType,
+        @Nullable IgniteThrowableBiConsumer<Integer, ByteBuffer> pagePostProcessor

Review Comment:
   Why is java's `BiConsumer` not enough?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054299721


##########
modules/compress/src/main/java/org/apache/ignite/internal/processors/compress/CompressionProcessorImpl.java:
##########
@@ -170,6 +177,28 @@ private ByteBuffer doCompactPage(ByteBuffer page, int pageSize) throws IgniteChe
         return compactPage;
     }
 
+    /** Check if filesystem actually supports punching holes. */
+    private void checkPunchHole(Path storagePath, int fsBlockSz) throws IgniteException {
+        Path testFile = storagePath.resolve("punch_hole_" + UUID.randomUUID() + ".test");
+
+        ByteBuffer buffer = GridUnsafe.allocateBuffer(fsBlockSz * 2);
+        GridUnsafe.zeroMemory(GridUnsafe.bufferAddress(buffer), buffer.capacity());
+
+        try {
+            try (RandomAccessFileIO testFileIO = new RandomAccessFileIO(testFile.toFile(), CREATE, WRITE)) {
+                testFileIO.writeFully(buffer);
+
+                testFileIO.punchHole(fsBlockSz, fsBlockSz);
+            }
+            finally {
+                Files.deleteIfExists(testFile);
+            }
+        }
+        catch (Exception e) {

Review Comment:
   It cannot be thrown here, it may be thrown at static initializer



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054305462


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -2670,6 +2738,21 @@ private boolean readPageFromStore(long pageId, ByteBuffer buff) throws IgniteChe
 
             assert read : toDetailString(pageId);
 
+            if (PageIO.getCompressionType(buff) != CompressionProcessor.UNCOMPRESSED_PAGE) {
+                int comprPageSz = PageIO.getCompressedSize(buff);
+
+                compressProc.decompressPage(buff, store.getPageSize());
+
+                if (punchHoleEnabled && comprPageSz < store.getPageSize()) {
+                    try {
+                        store.punchHole(pageId, comprPageSz);
+                    }
+                    catch (Exception ignored) {

Review Comment:
   Why we should ignore it?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] NSAmelchev commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
NSAmelchev commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054257562


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/SnapshotMetadata.java:
##########
@@ -191,6 +215,14 @@ private void writeObject(java.io.ObjectOutputStream s)
             for (Integer partId : e.getValue())
                 s.writeInt(partId);
         }
+
+        if (hasComprGrps) {
+            s.writeInt(comprGrpIds.size());
+
+            for (int grpId : comprGrpIds)
+                s.writeInt(grpId);
+

Review Comment:
   redundant line break



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054306440


##########
modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteSnapshotManager.java:
##########
@@ -2670,6 +2738,21 @@ private boolean readPageFromStore(long pageId, ByteBuffer buff) throws IgniteChe
 
             assert read : toDetailString(pageId);
 
+            if (PageIO.getCompressionType(buff) != CompressionProcessor.UNCOMPRESSED_PAGE) {
+                int comprPageSz = PageIO.getCompressedSize(buff);
+
+                compressProc.decompressPage(buff, store.getPageSize());
+
+                if (punchHoleEnabled && comprPageSz < store.getPageSize()) {
+                    try {
+                        store.punchHole(pageId, comprPageSz);
+                    }
+                    catch (Exception ignored) {

Review Comment:
   Because there is no need to fail here, punching holes is optional.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [ignite] ivandasch commented on a diff in pull request #10430: IGNITE-18198 Implement snapshots of caches with disk page compression.

Posted by GitBox <gi...@apache.org>.
ivandasch commented on code in PR #10430:
URL: https://github.com/apache/ignite/pull/10430#discussion_r1054324304


##########
modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/snapshot/IgniteClusterSnapshotDeltaTest.java:
##########
@@ -77,7 +80,8 @@ public static Collection<Object[]> parameters() {
     @Test
     public void testSendDelta() throws Exception {
         int keys = 10_000;
-        byte[] payload = new byte[DFLT_PAGE_SIZE / 2];
+        int pageSz = IgniteSystemProperties.getInteger(IGNITE_DEFAULT_DATA_STORAGE_PAGE_SIZE, DFLT_PAGE_SIZE);

Review Comment:
   No, it cannot be used here, because PAGE_SIZE doesn't contain valid value



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org