You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2021/04/21 15:28:56 UTC

[ignite] branch ignite-ducktape updated: IGNITE-14601 Specs should use service's params instead of copying. (#9027)

This is an automated email from the ASF dual-hosted git repository.

av pushed a commit to branch ignite-ducktape
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/ignite-ducktape by this push:
     new 09050ab  IGNITE-14601 Specs should use service's params instead of copying. (#9027)
09050ab is described below

commit 09050abb157395f2031f0f9cd4acc76628586425
Author: Anton Vinogradov <av...@apache.org>
AuthorDate: Wed Apr 21 18:28:20 2021 +0300

    IGNITE-14601 Specs should use service's params instead of copying. (#9027)
---
 .../ducktests/tests/ignitetest/services/ignite.py  |   2 +-
 .../tests/ignitetest/services/ignite_app.py        |   3 +-
 .../ignitetest/services/utils/ignite_aware.py      |   7 +-
 .../tests/ignitetest/services/utils/ignite_spec.py | 180 ++++++++++-----------
 .../tests/ignitetest/services/utils/jmx_utils.py   |  10 +-
 .../tests/ignitetest/tests/snapshot_test.py        |  50 ++----
 6 files changed, 106 insertions(+), 146 deletions(-)

diff --git a/modules/ducktests/tests/ignitetest/services/ignite.py b/modules/ducktests/tests/ignitetest/services/ignite.py
index 659739c..1e926c7 100644
--- a/modules/ducktests/tests/ignitetest/services/ignite.py
+++ b/modules/ducktests/tests/ignitetest/services/ignite.py
@@ -30,4 +30,4 @@ class IgniteService(IgniteAwareService):
     def __init__(self, context, config, num_nodes, jvm_opts=None, full_jvm_opts=None, startup_timeout_sec=60,
                  shutdown_timeout_sec=10, modules=None):
         super().__init__(context, config, num_nodes, startup_timeout_sec, shutdown_timeout_sec, self.APP_SERVICE_CLASS,
-                         modules=modules, jvm_opts=jvm_opts, full_jvm_opts=full_jvm_opts)
+                         modules, jvm_opts=jvm_opts, full_jvm_opts=full_jvm_opts)
diff --git a/modules/ducktests/tests/ignitetest/services/ignite_app.py b/modules/ducktests/tests/ignitetest/services/ignite_app.py
index 59e87ff..fa9f765 100644
--- a/modules/ducktests/tests/ignitetest/services/ignite_app.py
+++ b/modules/ducktests/tests/ignitetest/services/ignite_app.py
@@ -40,8 +40,7 @@ class IgniteApplicationService(IgniteAwareService):
                  shutdown_timeout_sec=10, modules=None, main_java_class=SERVICE_JAVA_CLASS_NAME, jvm_opts=None,
                  full_jvm_opts=None):
         super().__init__(context, config, num_nodes, startup_timeout_sec, shutdown_timeout_sec, main_java_class,
-                         modules=modules, java_class_name=java_class_name, params=params,
-                         jvm_opts=jvm_opts, full_jvm_opts=full_jvm_opts)
+                         modules, jvm_opts=jvm_opts, full_jvm_opts=full_jvm_opts)
 
         self.java_class_name = java_class_name
         self.params = params
diff --git a/modules/ducktests/tests/ignitetest/services/utils/ignite_aware.py b/modules/ducktests/tests/ignitetest/services/utils/ignite_aware.py
index ca906fd..10fbf54 100644
--- a/modules/ducktests/tests/ignitetest/services/utils/ignite_aware.py
+++ b/modules/ducktests/tests/ignitetest/services/utils/ignite_aware.py
@@ -54,7 +54,7 @@ class IgniteAwareService(BackgroundThreadService, IgnitePathAware, metaclass=ABC
         ALL = 2
 
     # pylint: disable=R0913
-    def __init__(self, context, config, num_nodes, startup_timeout_sec, shutdown_timeout_sec, main_java_class,
+    def __init__(self, context, config, num_nodes, startup_timeout_sec, shutdown_timeout_sec, main_java_class, modules,
                  **kwargs):
         """
         **kwargs are params that passed to IgniteSpec
@@ -69,8 +69,9 @@ class IgniteAwareService(BackgroundThreadService, IgnitePathAware, metaclass=ABC
         self.main_java_class = main_java_class
         self.startup_timeout_sec = startup_timeout_sec
         self.shutdown_timeout_sec = shutdown_timeout_sec
+        self.modules = modules
 
-        self.spec = resolve_spec(self, context, config, main_java_class, **kwargs)
+        self.spec = resolve_spec(self, **kwargs)
         self.init_logs_attribute()
 
         self.disconnected_nodes = []
@@ -113,7 +114,7 @@ class IgniteAwareService(BackgroundThreadService, IgnitePathAware, metaclass=ABC
 
         wait_until(lambda: self.alive(node), timeout_sec=10)
 
-        ignite_jmx_mixin(node, self.spec, self.pids(node))
+        ignite_jmx_mixin(node, self)
 
     def stop_async(self, force_stop=False, **kwargs):
         """
diff --git a/modules/ducktests/tests/ignitetest/services/utils/ignite_spec.py b/modules/ducktests/tests/ignitetest/services/utils/ignite_spec.py
index 28f309c..9731e2c 100644
--- a/modules/ducktests/tests/ignitetest/services/utils/ignite_spec.py
+++ b/modules/ducktests/tests/ignitetest/services/utils/ignite_spec.py
@@ -31,13 +31,14 @@ from ignitetest.services.utils.path import get_home_dir, get_module_path, Ignite
 from ignitetest.utils.version import DEV_BRANCH
 
 
-def resolve_spec(service, context, config, main_java_class, **kwargs):
+def resolve_spec(service, **kwargs):
     """
     Resolve Spec classes for IgniteService and IgniteApplicationService
     """
+
     def _resolve_spec(name, default):
-        if name in context.globals:
-            fqdn = context.globals[name]
+        if name in service.context.globals:
+            fqdn = service.context.globals[name]
             (module, clazz) = fqdn.rsplit('.', 1)
             module = importlib.import_module(module)
             return getattr(module, clazz)
@@ -49,26 +50,30 @@ def resolve_spec(service, context, config, main_java_class, **kwargs):
         return len(impl_filter) > 0
 
     if is_impl("IgniteService"):
-        return _resolve_spec("NodeSpec", ApacheIgniteNodeSpec)(path_aware=service, context=context, config=config,
-                                                               **kwargs)
+        return _resolve_spec("NodeSpec", IgniteNodeSpec)(service=service, **kwargs)
 
     if is_impl("IgniteApplicationService"):
-        return _resolve_spec("AppSpec", ApacheIgniteApplicationSpec)(path_aware=service, context=context, config=config,
-                                                                     main_java_class=main_java_class, **kwargs)
+        return _resolve_spec("AppSpec", IgniteApplicationSpec)(service=service, **kwargs)
 
     raise Exception("There is no specification for class %s" % type(service))
 
 
+def envs_to_exports(envs):
+    """
+    :return: line with exports env variables: export A=B; export C=D;
+    """
+    exports = ["export %s=%s" % (key, envs[key]) for key in envs]
+    return "; ".join(exports) + ";"
+
+
 class IgniteSpec(metaclass=ABCMeta):
     """
     This class is a basic Spec
     """
 
     # pylint: disable=R0913
-    def __init__(self, path_aware, config, jvm_opts=None, full_jvm_opts=None):
-        self.config = config
-        self.path_aware = path_aware
-        self.envs = {}
+    def __init__(self, service, jvm_opts, full_jvm_opts):
+        self.service = service
 
         if full_jvm_opts:
             self.jvm_opts = full_jvm_opts
@@ -77,22 +82,26 @@ class IgniteSpec(metaclass=ABCMeta):
                 self._add_jvm_opts(jvm_opts)
         else:
             self.jvm_opts = create_jvm_settings(opts=jvm_opts,
-                                                gc_dump_path=os.path.join(path_aware.log_dir, "ignite_gc.log"),
-                                                oom_path=os.path.join(path_aware.log_dir, "ignite_out_of_mem.hprof"))
+                                                gc_dump_path=os.path.join(service.log_dir, "ignite_gc.log"),
+                                                oom_path=os.path.join(service.log_dir, "ignite_out_of_mem.hprof"))
+
+        self._add_jvm_opts(["-DIGNITE_SUCCESS_FILE=" + os.path.join(self.service.persistent_root, "success_file"),
+                            "-Dlog4j.configuration=file:" + self.service.log_config_file,
+                            "-Dlog4j.configDebug=true"])
 
     @property
     def config_templates(self):
         """
         :return: config that service will use to start on a node
         """
-        if self.config.service_type == IgniteServiceType.NONE:
+        if self.service.config.service_type == IgniteServiceType.NONE:
             return []
 
         config_templates = [(IgnitePathAware.IGNITE_LOG_CONFIG_NAME, IgniteLoggerConfigTemplate())]
 
-        if self.config.service_type == IgniteServiceType.NODE:
+        if self.service.config.service_type == IgniteServiceType.NODE:
             config_templates.append((IgnitePathAware.IGNITE_CONFIG_NAME,
-                                     IgniteClientConfigTemplate() if self.config.client_mode
+                                     IgniteClientConfigTemplate() if self.service.config.client_mode
                                      else IgniteServerConfigTemplate()))
         else:
             config_templates.append((IgnitePathAware.IGNITE_THIN_CLIENT_CONFIG_NAME, IgniteThinClientConfigTemplate()))
@@ -103,8 +112,8 @@ class IgniteSpec(metaclass=ABCMeta):
         """
         Get home directory for current spec.
         """
-        product = product if product else str(self.config.version)
-        return get_home_dir(self.path_aware.install_root, product)
+        product = product if product else self.service.product
+        return get_home_dir(self.service.install_root, product)
 
     def _module(self, name):
         """
@@ -113,7 +122,7 @@ class IgniteSpec(metaclass=ABCMeta):
         if name == "ducktests":
             return get_module_path(self.__home(str(DEV_BRANCH)), name, DEV_BRANCH.is_dev)
 
-        return get_module_path(self.__home(), name, self.config.version.is_dev)
+        return get_module_path(self.__home(), name, self.service.config.version.is_dev)
 
     @abstractmethod
     def command(self, node):
@@ -121,12 +130,32 @@ class IgniteSpec(metaclass=ABCMeta):
         :return: string that represents command to run service on a node
         """
 
-    def _envs(self):
+    def libs(self):
         """
-        :return: line with exports env variables: export A=B; export C=D;
+        :return: libs set.
         """
-        exports = ["export %s=%s" % (key, self.envs[key]) for key in self.envs]
-        return "; ".join(exports) + ";"
+        libs = self.service.modules or []
+
+        libs.append("log4j")
+        libs.append("ducktests")
+
+        return list(map(lambda m: os.path.join(self._module(m), "*"), libs))
+
+    def envs(self):
+        """
+        :return: environment set.
+        """
+        return {
+            'EXCLUDE_TEST_CLASSES': 'true',
+            'IGNITE_LOG_DIR': self.service.persistent_root,
+            'USER_LIBS': ":".join(self.libs())
+        }
+
+    def config_file_path(self):
+        """
+        :return: path to project configuration file
+        """
+        return self.service.config_file
 
     def _jvm_opts(self):
         """
@@ -144,12 +173,13 @@ class IgniteNodeSpec(IgniteSpec):
     """
     Spec to run ignite node
     """
+
     def command(self, node):
         cmd = "%s %s %s %s 2>&1 | tee -a %s &" % \
-              (self._envs(),
-               self.path_aware.script("ignite.sh"),
+              (envs_to_exports(self.envs()),
+               self.service.script("ignite.sh"),
                self._jvm_opts(),
-               self.path_aware.config_file,
+               self.config_file_path(),
                node.log_file)
 
         return cmd
@@ -159,94 +189,48 @@ class IgniteApplicationSpec(IgniteSpec):
     """
     Spec to run ignite application
     """
+
     def __init__(self, **kwargs):
         super().__init__(**kwargs)
-        self.args = ""
-
-    def _app_args(self):
-        return ",".join(self.args)
+        self._add_jvm_opts(["-DIGNITE_NO_SHUTDOWN_HOOK=true",  # allows to perform operations on app termination.
+                            "-Xmx1G",
+                            "-ea",
+                            "-DIGNITE_ALLOW_ATOMIC_OPS_IN_TX=false"])
 
     def command(self, node):
+        args = [
+            str(self.service.config.service_type.name),
+            self.service.java_class_name,
+            self.config_file_path(),
+            str(base64.b64encode(json.dumps(self.service.params).encode('utf-8')), 'utf-8')
+        ]
+
         cmd = "%s %s %s %s 2>&1 | tee -a %s &" % \
-              (self._envs(),
-               self.path_aware.script("ignite.sh"),
+              (envs_to_exports(self.envs()),
+               self.service.script("ignite.sh"),
                self._jvm_opts(),
-               self._app_args(),
+               ",".join(args),
                node.log_file)
 
         return cmd
 
+    def config_file_path(self):
+        return self.service.config_file if self.service.config.service_type == IgniteServiceType.NODE \
+            else self.service.thin_client_config_file
 
-class ApacheIgniteNodeSpec(IgniteNodeSpec):
-    """
-    Implementation IgniteNodeSpec for Apache Ignite project
-    """
-    def __init__(self, context, modules, **kwargs):
-        super().__init__(**kwargs)
-        self.context = context
-
-        libs = (modules or [])
-        libs.append("log4j")
-        libs = list(map(lambda m: os.path.join(self._module(m), "*"), libs))
-
-        libs.append(os.path.join(self._module("ducktests"), "*"))
-
-        self.envs = {
-            'EXCLUDE_TEST_CLASSES': 'true',
-            'IGNITE_LOG_DIR': self.path_aware.persistent_root,
-            'USER_LIBS': ":".join(libs)
-        }
-
-        self._add_jvm_opts(["-DIGNITE_SUCCESS_FILE=" + os.path.join(self.path_aware.persistent_root, "success_file"),
-                            "-Dlog4j.configuration=file:" + self.path_aware.log_config_file,
-                            "-Dlog4j.configDebug=true"])
-
-
-class ApacheIgniteApplicationSpec(IgniteApplicationSpec):
-    """
-    Implementation IgniteApplicationSpec for Apache Ignite project
-    """
-    # pylint: disable=too-many-arguments
-    def __init__(self, context, modules, main_java_class, java_class_name, params, **kwargs):
-        super().__init__(**kwargs)
-        self.context = context
-
-        libs = modules or []
-        libs.extend(["log4j"])
-
-        libs = list(map(lambda m: os.path.join(self._module(m), "*"), libs))
-        libs.append(os.path.join(self._module("ducktests"), "*"))
+    def libs(self):
+        libs = super().libs()
         libs.extend(self.__jackson())
 
-        self.envs = {
-            "MAIN_CLASS": main_java_class,
-            "EXCLUDE_TEST_CLASSES": "true",
-            "IGNITE_LOG_DIR": self.path_aware.persistent_root,
-            "USER_LIBS": ":".join(libs)
-        }
-
-        self._add_jvm_opts(["-DIGNITE_SUCCESS_FILE=" + os.path.join(self.path_aware.persistent_root, "success_file"),
-                            "-Dlog4j.configuration=file:" + self.path_aware.log_config_file,
-                            "-Dlog4j.configDebug=true",
-                            "-DIGNITE_NO_SHUTDOWN_HOOK=true",  # allows to perform operations on app termination.
-                            "-Xmx1G",
-                            "-ea",
-                            "-DIGNITE_ALLOW_ATOMIC_OPS_IN_TX=false"])
-
-        config_file = self.path_aware.config_file if self.config.service_type == IgniteServiceType.NODE \
-            else self.path_aware.thin_client_config_file
-
-        self.args = [
-            str(self.config.service_type.name),
-            java_class_name,
-            config_file,
-            str(base64.b64encode(json.dumps(params).encode('utf-8')), 'utf-8')
-        ]
+        return libs
 
     def __jackson(self):
-        if not self.config.version.is_dev:
+        if not self.service.config.version.is_dev:
             aws = self._module("aws")
-            return self.context.cluster.nodes[0].account.ssh_capture(
+            return self.service.context.cluster.nodes[0].account.ssh_capture(
                 "ls -d %s/* | grep jackson | tr '\n' ':' | sed 's/.$//'" % aws)
 
         return []
+
+    def envs(self):
+        return {**super().envs(), **{"MAIN_CLASS": self.service.main_java_class}}
diff --git a/modules/ducktests/tests/ignitetest/services/utils/jmx_utils.py b/modules/ducktests/tests/ignitetest/services/utils/jmx_utils.py
index 1683938..ee6239e 100644
--- a/modules/ducktests/tests/ignitetest/services/utils/jmx_utils.py
+++ b/modules/ducktests/tests/ignitetest/services/utils/jmx_utils.py
@@ -23,14 +23,14 @@ import re
 from ignitetest.services.utils.decorators import memoize
 
 
-def ignite_jmx_mixin(node, spec, pids):
+def ignite_jmx_mixin(node, service):
     """
     Dynamically mixin JMX attributes to Ignite service node.
     :param node: Ignite service node.
-    :param pids: Ignite service node pids.
+    :param service: Ignite service.
     """
-    setattr(node, 'pids', pids)
-    setattr(node, 'spec', spec)
+    setattr(node, 'pids', service.pids(node))
+    setattr(node, 'install_root', service.install_root)
     base_cls = node.__class__
     base_cls_name = node.__class__.__name__
     node.__class__ = type(base_cls_name, (base_cls, IgniteJmxMixin), {})
@@ -58,7 +58,7 @@ class JmxClient:
     """
     def __init__(self, node):
         self.node = node
-        self.install_root = node.spec.path_aware.install_root
+        self.install_root = node.install_root
         self.pid = node.pids[0]
 
     @property
diff --git a/modules/ducktests/tests/ignitetest/tests/snapshot_test.py b/modules/ducktests/tests/ignitetest/tests/snapshot_test.py
index dbe176e..01917ca 100644
--- a/modules/ducktests/tests/ignitetest/tests/snapshot_test.py
+++ b/modules/ducktests/tests/ignitetest/tests/snapshot_test.py
@@ -53,69 +53,45 @@ class SnapshotTest(IgniteTest):
             metric_exporter='org.apache.ignite.spi.metric.jmx.JmxMetricExporterSpi'
         )
 
-        service = IgniteService(self.test_context, ignite_config, num_nodes=len(self.test_context.cluster) - 1)
-        service.start()
+        nodes = IgniteService(self.test_context, ignite_config, num_nodes=len(self.test_context.cluster) - 1)
+        nodes.start()
 
-        control_utility = ControlUtility(service)
+        control_utility = ControlUtility(nodes)
         control_utility.activate()
 
-        client_config = IgniteConfiguration(
-            client_mode=True,
-            version=version,
-            discovery_spi=from_ignite_cluster(service)
-        )
+        loader_config = IgniteConfiguration(client_mode=True, version=version, discovery_spi=from_ignite_cluster(nodes))
 
         loader = IgniteApplicationService(
             self.test_context,
-            client_config,
+            loader_config,
             java_class_name="org.apache.ignite.internal.ducktest.tests.snapshot_test.DataLoaderApplication",
-            params={
-                "start": 0,
-                "cacheName": self.CACHE_NAME,
-                "interval": 500_000,
-                "valueSizeKb": 1
-            }
+            params={"start": 0, "cacheName": self.CACHE_NAME, "interval": 500_000, "valueSizeKb": 1}
         )
 
         loader.run()
-        loader.free()
 
         control_utility.validate_indexes()
         control_utility.idle_verify()
-        node = service.nodes[0]
+
+        node = nodes.nodes[0]
 
         dump_1 = control_utility.idle_verify_dump(node)
 
         control_utility.snapshot_create(self.SNAPSHOT_NAME)
 
-        loader = IgniteApplicationService(
-            self.test_context,
-            client_config,
-            java_class_name="org.apache.ignite.internal.ducktest.tests.snapshot_test.DataLoaderApplication",
-            params={
-                "start": 500_000,
-                "cacheName": self.CACHE_NAME,
-                "interval": 100_000,
-                "valueSizeKb": 1
-            }
-        )
-
-        loader.start(clean=False)
-        loader.wait()
+        loader.params = {"start": 500_000, "cacheName": self.CACHE_NAME, "interval": 100_000, "valueSizeKb": 1}
+        loader.run()
 
         dump_2 = control_utility.idle_verify_dump(node)
 
         diff = node.account.ssh_output(f'diff {dump_1} {dump_2}', allow_fail=True)
         assert len(diff) != 0
 
-        service.stop()
-
-        service.restore_from_snapshot(self.SNAPSHOT_NAME)
-
-        service.start(clean=False)
+        nodes.stop()
+        nodes.restore_from_snapshot(self.SNAPSHOT_NAME)
+        nodes.start(clean=False)
 
         control_utility.activate()
-
         control_utility.validate_indexes()
         control_utility.idle_verify()