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 2021/04/05 13:40:45 UTC

[GitHub] [ignite] map7000 opened a new pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

map7000 opened a new pull request #8971:
URL: https://github.com/apache/ignite/pull/8971


   A simple test to check PDS compatibility of different Ignite versions
   
   Ignite versions "from_version" and "to_version" are set via test parameters
   
       Start Ignite cluster version "from_version" with PDS enabled
       Start a client application that puts prepared data looks like
       User (1, "John Connor")
       User (2, "Sarah Connor")
       User (3, "Kyle Reese")
       Stop cluster and client
       Start Ignite cluster version "to_version" without PDS clearing
       Start client that reads data and checks that it can be read and have not changed
   
   
   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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607103925



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"
+    CHECK_OPERATION = "checkData"
+
+    @cluster(num_nodes=3)
+    @versions_pair(str(LATEST), str(DEV_BRANCH))
+    def test_pds_compatibility(self, ignite_version_1, ignite_version_2):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1
+
+        server_configuration_1 = IgniteConfiguration(version=IgniteVersion(ignite_version_1),
+                                                     caches=[
+                                                         CacheConfiguration(name=self.CACHE_NAME, backups=1,
+                                                                            atomicity_mode='ATOMIC')],
+                                                     data_storage=DataStorageConfiguration(
+                                                         default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_2 = server_configuration_1._replace(version=IgniteVersion(ignite_version_2))
+
+        ignite_1 = IgniteService(self.test_context, server_configuration_1, num_nodes=num_nodes)
+        nodes = ignite_1.nodes.copy()
+
+        ignite_1.start()
+
+        self._run_application(ignite_1, self.LOAD_OPERATION)
+
+        ignite_1.stop()
+        ignite_1.free()
+
+        ignite_2 = IgniteService(self.test_context, server_configuration_2, num_nodes=num_nodes)
+        ignite_2.nodes = nodes
+        ignite_2.start(clean=False)
+
+        self._run_application(ignite_2, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"cacheName": self.CACHE_NAME, "operation": operation})
+        app.start()
+        app.stop()

Review comment:
       We should use `await_stopped` check to ensure the application stoped successfully.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607177185



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,94 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")

Review comment:
       Please, remove data list from test description.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607102429



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) {
+
+        String operation = jsonNode.get("operation").asText();
+        String cacheName = jsonNode.get("cacheName").asText();

Review comment:
       let's use constant cache name




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607104274



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"
+    CHECK_OPERATION = "checkData"
+
+    @cluster(num_nodes=3)
+    @versions_pair(str(LATEST), str(DEV_BRANCH))
+    def test_pds_compatibility(self, ignite_version_1, ignite_version_2):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1
+
+        server_configuration_1 = IgniteConfiguration(version=IgniteVersion(ignite_version_1),
+                                                     caches=[
+                                                         CacheConfiguration(name=self.CACHE_NAME, backups=1,
+                                                                            atomicity_mode='ATOMIC')],
+                                                     data_storage=DataStorageConfiguration(
+                                                         default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_2 = server_configuration_1._replace(version=IgniteVersion(ignite_version_2))
+
+        ignite_1 = IgniteService(self.test_context, server_configuration_1, num_nodes=num_nodes)
+        nodes = ignite_1.nodes.copy()
+
+        ignite_1.start()
+
+        self._run_application(ignite_1, self.LOAD_OPERATION)
+
+        ignite_1.stop()
+        ignite_1.free()
+
+        ignite_2 = IgniteService(self.test_context, server_configuration_2, num_nodes=num_nodes)
+        ignite_2.nodes = nodes
+        ignite_2.start(clean=False)
+
+        self._run_application(ignite_2, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"cacheName": self.CACHE_NAME, "operation": operation})
+        app.start()
+        app.stop()
+        app.free()

Review comment:
       Why do we need to call `free` and `decativate` manually?
   It seems all will work just with automatic stop.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607095652



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");

Review comment:
       this can be private.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607173365



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) throws IgniteCheckedException {
+
+        String operation = jsonNode.get("operation").asText();
+        final String cacheName = "users";
+
+        markInitialized();
+
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
+
+        log.info("Input data: " + jsonNode.toString());

Review comment:
       Please, remove useless log.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607102321



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) {
+
+        String operation = jsonNode.get("operation").asText();
+        String cacheName = jsonNode.get("cacheName").asText();
+
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
+
+        if (operation.equals("load")) {
+            for (int i = 0; i < users.size(); i++) {
+                cache.put(i, users.get(i));
+            }
+        } else if (operation.equals("check")) {

Review comment:
       else should be on the new line.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607175147



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) throws IgniteCheckedException {
+
+        String operation = jsonNode.get("operation").asText();
+        final String cacheName = "users";
+
+        markInitialized();
+
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
+
+        log.info("Input data: " + jsonNode.toString());
+
+        switch (operation) {
+            case "load":
+                for (int i = 0; i < users.size(); i++) {

Review comment:
       square bracket not required for one line loop.
   Please, remove 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607163503



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,93 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=3)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 2
+
+        server_configuration_from = IgniteConfiguration(version=IgniteVersion(version_from),
+                                                        data_storage=DataStorageConfiguration(
+                                                            default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_to = server_configuration_from._replace(version=IgniteVersion(version_to))
+
+        ignite_from = IgniteService(self.test_context, server_configuration_from, num_nodes=num_nodes)
+        nodes = ignite_from.nodes.copy()
+
+        ignite_from.start()
+
+        self._run_application(ignite_from, self.LOAD_OPERATION)
+
+        ignite_from.stop()
+        ignite_from.free()
+
+        ignite_to = IgniteService(self.test_context, server_configuration_to, num_nodes=num_nodes)
+        ignite_to.nodes = nodes
+        ignite_to.start(clean=False)
+
+        self._run_application(ignite_to, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"operation": operation})
+        app.start()
+        app.await_stopped()
+        control_utility.deactivate()

Review comment:
       Why do we need to `deactivate` cluster here?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607177269



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,94 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions

Review comment:
       dot in the end required.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607173049



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) throws IgniteCheckedException {
+
+        String operation = jsonNode.get("operation").asText();
+        final String cacheName = "users";
+
+        markInitialized();
+
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
+
+        log.info("Input data: " + jsonNode.toString());
+
+        switch (operation) {
+            case "load":
+                for (int i = 0; i < users.size(); i++) {
+                    cache.put(i, users.get(i));
+                }
+                break;

Review comment:
       empty line required.

##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) throws IgniteCheckedException {
+

Review comment:
       please, remove empty line.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607099301



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"
+    CHECK_OPERATION = "checkData"
+
+    @cluster(num_nodes=3)
+    @versions_pair(str(LATEST), str(DEV_BRANCH))
+    def test_pds_compatibility(self, ignite_version_1, ignite_version_2):

Review comment:
       Please, rename to version_from, version_to




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607102620



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {

Review comment:
       Let's rename this to `PdsCompatiblityApplication`




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

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



[GitHub] [ignite] map7000 commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
map7000 commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607188220



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,93 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=3)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 2
+
+        server_configuration_from = IgniteConfiguration(version=IgniteVersion(version_from),
+                                                        data_storage=DataStorageConfiguration(
+                                                            default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_to = server_configuration_from._replace(version=IgniteVersion(version_to))
+
+        ignite_from = IgniteService(self.test_context, server_configuration_from, num_nodes=num_nodes)
+        nodes = ignite_from.nodes.copy()
+
+        ignite_from.start()
+
+        self._run_application(ignite_from, self.LOAD_OPERATION)
+
+        ignite_from.stop()
+        ignite_from.free()
+
+        ignite_to = IgniteService(self.test_context, server_configuration_to, num_nodes=num_nodes)
+        ignite_to.nodes = nodes
+        ignite_to.start(clean=False)
+
+        self._run_application(ignite_to, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"operation": operation})
+        app.start()
+        app.await_stopped()
+        control_utility.deactivate()

Review comment:
       We stop cluster using kill command. 
   I don't know that kind of side-effects it can bring us
   It's safe to deactivate first




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

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



[GitHub] [ignite] map7000 commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
map7000 commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607204620



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,93 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=3)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 2
+
+        server_configuration_from = IgniteConfiguration(version=IgniteVersion(version_from),
+                                                        data_storage=DataStorageConfiguration(
+                                                            default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_to = server_configuration_from._replace(version=IgniteVersion(version_to))
+
+        ignite_from = IgniteService(self.test_context, server_configuration_from, num_nodes=num_nodes)
+        nodes = ignite_from.nodes.copy()
+
+        ignite_from.start()
+
+        self._run_application(ignite_from, self.LOAD_OPERATION)
+
+        ignite_from.stop()
+        ignite_from.free()
+
+        ignite_to = IgniteService(self.test_context, server_configuration_to, num_nodes=num_nodes)
+        ignite_to.nodes = nodes
+        ignite_to.start(clean=False)
+
+        self._run_application(ignite_to, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"operation": operation})
+        app.start()
+        app.await_stopped()
+        control_utility.deactivate()

Review comment:
       ok. i will remove deactivation here




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607547007



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=2)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1

Review comment:
       Can we just user `1` here and remove this variable.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=2)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1

Review comment:
       Can we just user `1` here and remove this variable?

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=2)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1

Review comment:
       Can we just use `1` here and remove this variable?




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607173166



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,68 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) throws IgniteCheckedException {
+
+        String operation = jsonNode.get("operation").asText();
+        final String cacheName = "users";

Review comment:
       please, inline cache name.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607096988



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) {
+
+        String operation = jsonNode.get("operation").asText();
+        String cacheName = jsonNode.get("cacheName").asText();
+
+        IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName);
+
+        if (operation.equals("load")) {
+            for (int i = 0; i < users.size(); i++) {
+                cache.put(i, users.get(i));
+            }
+        } else if (operation.equals("check")) {
+            for (int i = 0; i < users.size(); i++) {
+                assert cache.get(i).equals(users.get(i));
+            }
+        }

Review comment:
       please, add an exception in case of unexpected operation




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607096180



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");
+
+    /** {@inheritDoc} */
+    @Override protected void run(JsonNode jsonNode) {
+
+        String operation = jsonNode.get("operation").asText();

Review comment:
       please, add markInitialized, markFinished




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

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



[GitHub] [ignite] nizhikov merged pull request #8971: IGNITE-13605 Basic PDS compatibility test

Posted by GitBox <gi...@apache.org>.
nizhikov merged pull request #8971:
URL: https://github.com/apache/ignite/pull/8971


   


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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607545231



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache
+ * "check" - check if we have that predifined data in that cache
+ */
+public class PdsCompatiblityApplication extends IgniteAwareApplication {
+    /** Predefined test data. */
+    private static List<String> users = Arrays.asList("John Connor", "Sarah Connor", "Kyle Reese");

Review comment:
       Let's have custom model type(`User`) here to ensure the same type can be read with the new version.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled

Review comment:
       dot in the end.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data

Review comment:
       dot in the end.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client

Review comment:
       dot in the end.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing

Review comment:
       dot in the end.

##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,100 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions.
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed

Review comment:
       dot in the end.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607099556



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"

Review comment:
       Analog in the app named as `load`.
   Please, fix 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.

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607100710



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"
+    CHECK_OPERATION = "checkData"
+
+    @cluster(num_nodes=3)
+    @versions_pair(str(LATEST), str(DEV_BRANCH))

Review comment:
       Please, remove this annotation and use matrix instead.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607098043



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,98 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.services.utils.ignite_configuration.cache import CacheConfiguration
+from ignitetest.utils import cluster, versions_pair
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.DictionaryCacheApplication"
+    CACHE_NAME = "users"
+    LOAD_OPERATION = "loadData"
+    CHECK_OPERATION = "checkData"
+
+    @cluster(num_nodes=3)
+    @versions_pair(str(LATEST), str(DEV_BRANCH))
+    def test_pds_compatibility(self, ignite_version_1, ignite_version_2):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 1
+
+        server_configuration_1 = IgniteConfiguration(version=IgniteVersion(ignite_version_1),

Review comment:
       Please, remove all cache configuration and just create one from an application.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607095499



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/DictionaryCacheApplication.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+public class DictionaryCacheApplication extends IgniteAwareApplication {

Review comment:
       add javadoc, please.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607542212



##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options

Review comment:
       dot in the end.

##########
File path: modules/ducktests/src/main/java/org/apache/ignite/internal/ducktest/tests/compatibility/PdsCompatiblityApplication.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.ducktest.tests.compatibility;
+
+import java.util.Arrays;
+import java.util.List;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.ducktest.utils.IgniteAwareApplication;
+
+/**
+ * Simple application that have 2 options
+ * "load" - load some predefined data to cache

Review comment:
       dot in the end.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607101926



##########
File path: modules/ducktests/tests/ignitetest/utils/_mark.py
##########
@@ -54,6 +55,7 @@ class IgniteVersionParametrize(Mark):
     """
     Parametrize function with ignite_version
     """
+

Review comment:
       please, revert empty line changes.




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

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



[GitHub] [ignite] nizhikov commented on a change in pull request #8971: IGNITE-13605 PDS compatibility for ignite versions

Posted by GitBox <gi...@apache.org>.
nizhikov commented on a change in pull request #8971:
URL: https://github.com/apache/ignite/pull/8971#discussion_r607195021



##########
File path: modules/ducktests/tests/ignitetest/tests/compatibility/pds_compatibility_test.py
##########
@@ -0,0 +1,93 @@
+# 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.
+
+"""
+This module contains test that checks that PDS "from_version" compatible with "to_version"
+"""
+from ducktape.mark import parametrize
+
+from ignitetest.services.ignite import IgniteService
+from ignitetest.services.ignite_app import IgniteApplicationService
+from ignitetest.services.utils.control_utility import ControlUtility
+from ignitetest.services.utils.ignite_configuration.discovery import from_ignite_cluster
+from ignitetest.services.utils.ignite_configuration import IgniteConfiguration, DataStorageConfiguration
+from ignitetest.services.utils.ignite_configuration.data_storage import DataRegionConfiguration
+from ignitetest.utils import cluster
+from ignitetest.utils.ignite_test import IgniteTest
+from ignitetest.utils.version import DEV_BRANCH, LATEST, IgniteVersion
+
+
+# pylint: disable=W0223
+# pylint: disable=no-member
+class PdsCompatibilityTest(IgniteTest):
+    """
+    A simple test to check PDS compatibility of different Ignite versions
+
+    Start Ignite cluster version "from_version" with PDS enabled
+    Start a client application that puts prepared data looks like
+    User (1, "John Connor")
+    User (2, "Sarah Connor")
+    User (3, "Kyle Reese")
+    Stop cluster and client
+    Start Ignite cluster version "to_version" without PDS clearing
+    Start client that reads data and checks that it can be read and have not changed
+
+    """
+    APP_CLASS = "org.apache.ignite.internal.ducktest.tests.compatibility.PdsCompatiblityApplication"
+    LOAD_OPERATION = "load"
+    CHECK_OPERATION = "check"
+
+    @cluster(num_nodes=3)
+    @parametrize(version_from=str(LATEST), version_to=str(DEV_BRANCH))
+    def test_pds_compatibility(self, version_from, version_to):
+        """
+        Saves data using one version of ignite and then load with another.
+        """
+
+        num_nodes = len(self.test_context.cluster) - 2
+
+        server_configuration_from = IgniteConfiguration(version=IgniteVersion(version_from),
+                                                        data_storage=DataStorageConfiguration(
+                                                            default=DataRegionConfiguration(persistent=True)))
+
+        server_configuration_to = server_configuration_from._replace(version=IgniteVersion(version_to))
+
+        ignite_from = IgniteService(self.test_context, server_configuration_from, num_nodes=num_nodes)
+        nodes = ignite_from.nodes.copy()
+
+        ignite_from.start()
+
+        self._run_application(ignite_from, self.LOAD_OPERATION)
+
+        ignite_from.stop()
+        ignite_from.free()
+
+        ignite_to = IgniteService(self.test_context, server_configuration_to, num_nodes=num_nodes)
+        ignite_to.nodes = nodes
+        ignite_to.start(clean=False)
+
+        self._run_application(ignite_to, self.CHECK_OPERATION)
+
+    def _run_application(self, ignite, operation):
+        control_utility = ControlUtility(ignite)
+        control_utility.activate()
+
+        app_config = ignite.config._replace(client_mode=True, discovery_spi=from_ignite_cluster(ignite))
+        app = IgniteApplicationService(self.test_context, config=app_config,
+                                       java_class_name=self.APP_CLASS,
+                                       params={"operation": operation})
+        app.start()
+        app.await_stopped()
+        control_utility.deactivate()

Review comment:
       kill initiates graceful shutdown, isn't 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.

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