You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2017/02/16 17:36:25 UTC

[24/50] [abbrv] ignite git commit: IGNITE-4678 Web Console: Implemented demo load as service.

IGNITE-4678 Web Console: Implemented demo load as service.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a600cafd
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a600cafd
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a600cafd

Branch: refs/heads/ignite-comm-balance-master
Commit: a600cafd1eec7581e8edde5a10b7d171997551f7
Parents: 3bf880c
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Feb 10 15:55:05 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Feb 10 15:55:05 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js        |  34 ++
 modules/web-console/backend/app/browser.js      |  26 +
 modules/web-console/frontend/package.json       |   2 +-
 .../ignite/console/demo/AgentClusterDemo.java   | 475 +------------------
 .../ignite/console/demo/AgentDemoUtils.java     |  79 +++
 .../demo/service/DemoCachesLoadService.java     | 456 ++++++++++++++++++
 .../service/DemoRandomCacheLoadService.java     | 120 +++++
 .../service/DemoServiceClusterSingleton.java    |  41 ++
 .../demo/service/DemoServiceKeyAffinity.java    |  41 ++
 .../service/DemoServiceMultipleInstances.java   |  41 ++
 .../demo/service/DemoServiceNodeSingleton.java  |  41 ++
 11 files changed, 897 insertions(+), 459 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/backend/app/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js
index 8170280..4cae8ee 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -581,6 +581,40 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo,
 
             return this.executeRest(cmd);
         }
+
+        /**
+         * Collect service information.
+         * @param {Boolean} demo Is need run command on demo node.
+         * @param {String} nid Node ID.
+         * @returns {Promise}
+         */
+        services(demo, nid) {
+            const cmd = new Command(demo, 'exe')
+                .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask')
+                .addParam('p1', nid)
+                .addParam('p2', 'org.apache.ignite.internal.visor.service.VisorServiceTask')
+                .addParam('p3', 'java.lang.Void');
+
+            return this.executeRest(cmd);
+        }
+
+        /**
+         * Cancel service with specified name.
+         * @param {Boolean} demo Is need run command on demo node.
+         * @param {String} nid Node ID.
+         * @param {String} name Name of service to cancel.
+         * @returns {Promise}
+         */
+        serviceCancel(demo, nid, name) {
+            const cmd = new Command(demo, 'exe')
+                .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask')
+                .addParam('p1', nid)
+                .addParam('p2', 'org.apache.ignite.internal.visor.service.VisorCancelServiceTask')
+                .addParam('p3', 'java.lang.String')
+                .addParam('p4', name);
+
+            return this.executeRest(cmd);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/backend/app/browser.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js
index 499d84d..2b1285e 100644
--- a/modules/web-console/backend/app/browser.js
+++ b/modules/web-console/backend/app/browser.js
@@ -455,6 +455,32 @@ module.exports.factory = (_, socketio, agentMgr, configure) => {
                         .catch((err) => cb(_errorToJson(err)));
                 });
 
+                // Collect service information from grid.
+                socket.on('service:collect', (nid, cb) => {
+                    agentMgr.findAgent(accountId())
+                        .then((agent) => agent.services(demo, nid))
+                        .then((data) => {
+                            if (data.finished)
+                                return cb(null, data.result);
+
+                            cb(_errorToJson(data.error));
+                        })
+                        .catch((err) => cb(_errorToJson(err)));
+                });
+
+                // Collect service information from grid.
+                socket.on('service:cancel', (nid, name, cb) => {
+                    agentMgr.findAgent(accountId())
+                        .then((agent) => agent.serviceCancel(demo, nid, name))
+                        .then((data) => {
+                            if (data.finished)
+                                return cb(null, data.result);
+
+                            cb(_errorToJson(data.error));
+                        })
+                        .catch((err) => cb(_errorToJson(err)));
+                });
+
                 const count = agentMgr.addAgentListener(user._id, socket);
 
                 socket.emit('agent:count', {count});

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index ff52ba4..651f496 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -46,7 +46,7 @@
     "angular-touch": "~1.5.9",
     "angular-translate": "~2.13.1",
     "angular-tree-control": "~0.2.26",
-    "angular-ui-grid": "~3.2.9",
+    "angular-ui-grid": "~4.0.0",
     "angular-ui-router": "~0.3.1",
     "bootstrap-sass": "~3.3.6",
     "brace": "~0.8.0",

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
index 489e762..252692e 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
@@ -17,37 +17,24 @@
 
 package org.apache.ignite.console.demo;
 
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.Random;
-import java.util.Set;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteServices;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryIndex;
-import org.apache.ignite.cache.QueryIndexType;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.console.agent.AgentConfiguration;
-import org.apache.ignite.console.demo.model.Car;
-import org.apache.ignite.console.demo.model.Country;
-import org.apache.ignite.console.demo.model.Department;
-import org.apache.ignite.console.demo.model.Employee;
-import org.apache.ignite.console.demo.model.Parking;
+import org.apache.ignite.console.demo.service.DemoCachesLoadService;
+import org.apache.ignite.console.demo.service.DemoRandomCacheLoadService;
+import org.apache.ignite.console.demo.service.DemoServiceMultipleInstances;
+import org.apache.ignite.console.demo.service.DemoServiceClusterSingleton;
+import org.apache.ignite.console.demo.service.DemoServiceKeyAffinity;
+import org.apache.ignite.console.demo.service.DemoServiceNodeSingleton;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.logger.log4j.Log4JLogger;
@@ -55,7 +42,6 @@ import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi;
-import org.apache.ignite.transactions.Transaction;
 import org.apache.log4j.Logger;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
@@ -66,8 +52,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII;
 import static org.apache.ignite.events.EventType.EVTS_DISCOVERY;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_ADDRS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_JETTY_PORT;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * Demo for cluster features like SQL and Monitoring.
@@ -84,247 +68,6 @@ public class AgentClusterDemo {
     /** */
     private static final int NODE_CNT = 3;
 
-    /** */
-    private static final String COUNTRY_CACHE_NAME = "CountryCache";
-
-    /** */
-    private static final String DEPARTMENT_CACHE_NAME = "DepartmentCache";
-
-    /** */
-    private static final String EMPLOYEE_CACHE_NAME = "EmployeeCache";
-
-    /** */
-    private static final String PARKING_CACHE_NAME = "ParkingCache";
-
-    /** */
-    private static final String CAR_CACHE_NAME = "CarCache";
-
-    /** */
-    private static final Set<String> DEMO_CACHES = new HashSet<>(Arrays.asList(COUNTRY_CACHE_NAME,
-        DEPARTMENT_CACHE_NAME, EMPLOYEE_CACHE_NAME, PARKING_CACHE_NAME, CAR_CACHE_NAME));
-
-    /** */
-    private static final Random rnd = new Random();
-
-    /** Countries count. */
-    private static final int CNTR_CNT = 10;
-
-    /** Departments count */
-    private static final int DEP_CNT = 100;
-
-    /** Employees count. */
-    private static final int EMPL_CNT = 1000;
-
-    /** Countries count. */
-    private static final int CAR_CNT = 100;
-
-    /** Departments count */
-    private static final int PARK_CNT = 10;
-
-    /** Counter for threads in pool. */
-    private static final AtomicInteger THREAD_CNT = new AtomicInteger(0);
-
-    /**
-     * Create base cache configuration.
-     *
-     * @param name cache name.
-     * @return Cache configuration with basic properties set.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheConfiguration(String name) {
-        CacheConfiguration<K, V> ccfg = new CacheConfiguration<>(name);
-
-        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
-        ccfg.setQueryDetailMetricsSize(10);
-        ccfg.setStartSize(100);
-        ccfg.setStatisticsEnabled(true);
-
-        return ccfg;
-    }
-
-    /**
-     * Configure cacheCountry.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheCountry() {
-        CacheConfiguration<K, V> ccfg = cacheConfiguration(COUNTRY_CACHE_NAME);
-
-        // Configure cacheCountry types.
-        Collection<QueryEntity> qryEntities = new ArrayList<>();
-
-        // COUNTRY.
-        QueryEntity type = new QueryEntity();
-
-        qryEntities.add(type);
-
-        type.setKeyType(Integer.class.getName());
-        type.setValueType(Country.class.getName());
-
-        // Query fields for COUNTRY.
-        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
-
-        qryFlds.put("id", "java.lang.Integer");
-        qryFlds.put("name", "java.lang.String");
-        qryFlds.put("population", "java.lang.Integer");
-
-        type.setFields(qryFlds);
-
-        ccfg.setQueryEntities(qryEntities);
-
-        return ccfg;
-    }
-
-    /**
-     * Configure cacheEmployee.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheDepartment() {
-        CacheConfiguration<K, V> ccfg = cacheConfiguration(DEPARTMENT_CACHE_NAME);
-
-        // Configure cacheDepartment types.
-        Collection<QueryEntity> qryEntities = new ArrayList<>();
-
-        // DEPARTMENT.
-        QueryEntity type = new QueryEntity();
-
-        qryEntities.add(type);
-
-        type.setKeyType(Integer.class.getName());
-        type.setValueType(Department.class.getName());
-
-        // Query fields for DEPARTMENT.
-        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
-
-        qryFlds.put("id", "java.lang.Integer");
-        qryFlds.put("countryId", "java.lang.Integer");
-        qryFlds.put("name", "java.lang.String");
-
-        type.setFields(qryFlds);
-
-        ccfg.setQueryEntities(qryEntities);
-
-        return ccfg;
-    }
-
-    /**
-     * Configure cacheEmployee.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheEmployee() {
-        CacheConfiguration<K, V> ccfg = cacheConfiguration(EMPLOYEE_CACHE_NAME);
-
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setBackups(1);
-
-        // Configure cacheEmployee types.
-        Collection<QueryEntity> qryEntities = new ArrayList<>();
-
-        // EMPLOYEE.
-        QueryEntity type = new QueryEntity();
-
-        qryEntities.add(type);
-
-        type.setKeyType(Integer.class.getName());
-        type.setValueType(Employee.class.getName());
-
-        // Query fields for EMPLOYEE.
-        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
-
-        qryFlds.put("id", "java.lang.Integer");
-        qryFlds.put("departmentId", "java.lang.Integer");
-        qryFlds.put("managerId", "java.lang.Integer");
-        qryFlds.put("firstName", "java.lang.String");
-        qryFlds.put("lastName", "java.lang.String");
-        qryFlds.put("email", "java.lang.String");
-        qryFlds.put("phoneNumber", "java.lang.String");
-        qryFlds.put("hireDate", "java.sql.Date");
-        qryFlds.put("job", "java.lang.String");
-        qryFlds.put("salary", "java.lang.Double");
-
-        type.setFields(qryFlds);
-
-        // Indexes for EMPLOYEE.
-        Collection<QueryIndex> indexes = new ArrayList<>();
-
-        QueryIndex idx = new QueryIndex();
-
-        idx.setName("EMP_NAMES");
-        idx.setIndexType(QueryIndexType.SORTED);
-        LinkedHashMap<String, Boolean> indFlds = new LinkedHashMap<>();
-
-        indFlds.put("firstName", Boolean.FALSE);
-        indFlds.put("lastName", Boolean.FALSE);
-
-        idx.setFields(indFlds);
-
-        indexes.add(idx);
-        indexes.add(new QueryIndex("salary", QueryIndexType.SORTED, false, "EMP_SALARY"));
-
-        type.setIndexes(indexes);
-
-        ccfg.setQueryEntities(qryEntities);
-
-        return ccfg;
-    }
-
-    /**
-     * Configure cacheEmployee.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheParking() {
-        CacheConfiguration<K, V> ccfg = cacheConfiguration(PARKING_CACHE_NAME);
-
-        // Configure cacheParking types.
-        Collection<QueryEntity> qryEntities = new ArrayList<>();
-
-        // PARKING.
-        QueryEntity type = new QueryEntity();
-
-        qryEntities.add(type);
-
-        type.setKeyType(Integer.class.getName());
-        type.setValueType(Parking.class.getName());
-
-        // Query fields for PARKING.
-        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
-
-        qryFlds.put("id", "java.lang.Integer");
-        qryFlds.put("name", "java.lang.String");
-        qryFlds.put("capacity", "java.lang.Integer");
-
-        type.setFields(qryFlds);
-
-        ccfg.setQueryEntities(qryEntities);
-
-        return ccfg;
-    }
-
-    /**
-     * Configure cacheEmployee.
-     */
-    private static <K, V> CacheConfiguration<K, V> cacheCar() {
-        CacheConfiguration<K, V> ccfg = cacheConfiguration(CAR_CACHE_NAME);
-
-        // Configure cacheCar types.
-        Collection<QueryEntity> qryEntities = new ArrayList<>();
-
-        // CAR.
-        QueryEntity type = new QueryEntity();
-
-        qryEntities.add(type);
-
-        type.setKeyType(Integer.class.getName());
-        type.setValueType(Car.class.getName());
-
-        // Query fields for CAR.
-        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
-
-        qryFlds.put("id", "java.lang.Integer");
-        qryFlds.put("parkingId", "java.lang.Integer");
-        qryFlds.put("name", "java.lang.String");
-
-        type.setFields(qryFlds);
-
-        ccfg.setQueryEntities(qryEntities);
-
-        return ccfg;
-    }
-
     /**
      * Configure node.
      * @param gridIdx Grid name index.
@@ -363,212 +106,20 @@ public class AgentClusterDemo {
         if (client)
             cfg.setClientMode(true);
 
-        cfg.setCacheConfiguration(cacheCountry(), cacheDepartment(), cacheEmployee(), cacheParking(), cacheCar());
-
         cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
         return cfg;
     }
 
     /**
-     * @param val Value to round.
-     * @param places Numbers after point.
-     * @return Rounded value;
-     */
-    private static double round(double val, int places) {
-        if (places < 0)
-            throw new IllegalArgumentException();
-
-        long factor = (long)Math.pow(10, places);
-
-        val *= factor;
-
-        long tmp = Math.round(val);
-
-        return (double)tmp / factor;
-    }
-
-    /**
-     * @param ignite Ignite.
-     * @param range Time range in milliseconds.
-     */
-    private static void populateCacheEmployee(Ignite ignite, long range) {
-        if (log.isDebugEnabled())
-            log.debug("DEMO: Start employees population with data...");
-
-        IgniteCache<Integer, Country> cacheCountry = ignite.cache(COUNTRY_CACHE_NAME);
-
-        for (int i = 0, n = 1; i < CNTR_CNT; i++, n++)
-            cacheCountry.put(i, new Country(i, "Country #" + n, n * 10000000));
-
-        IgniteCache<Integer, Department> cacheDepartment = ignite.cache(DEPARTMENT_CACHE_NAME);
-
-        IgniteCache<Integer, Employee> cacheEmployee = ignite.cache(EMPLOYEE_CACHE_NAME);
-
-        for (int i = 0, n = 1; i < DEP_CNT; i++, n++) {
-            cacheDepartment.put(i, new Department(n, rnd.nextInt(CNTR_CNT), "Department #" + n));
-
-            double r = rnd.nextDouble();
-
-            cacheEmployee.put(i, new Employee(i, rnd.nextInt(DEP_CNT), null, "First name manager #" + n,
-                "Last name manager #" + n, "Email manager #" + n, "Phone number manager #" + n,
-                new java.sql.Date((long)(r * range)), "Job manager #" + n, 1000 + round(r * 4000, 2)));
-        }
-
-        for (int i = 0, n = 1; i < EMPL_CNT; i++, n++) {
-            Integer depId = rnd.nextInt(DEP_CNT);
-
-            double r = rnd.nextDouble();
-
-            cacheEmployee.put(i, new Employee(i, depId, depId, "First name employee #" + n,
-                "Last name employee #" + n, "Email employee #" + n, "Phone number employee #" + n,
-                new java.sql.Date((long)(r * range)), "Job employee #" + n, 500 + round(r * 2000, 2)));
-        }
-
-        if (log.isDebugEnabled())
-            log.debug("DEMO: Finished employees population.");
-    }
-
-    /**
-     * @param ignite Ignite.
-     */
-    private static void populateCacheCar(Ignite ignite) {
-        if (log.isDebugEnabled())
-            log.debug("DEMO: Start cars population...");
-
-        IgniteCache<Integer, Parking> cacheParking = ignite.cache(PARKING_CACHE_NAME);
-
-        for (int i = 0, n = 1; i < PARK_CNT; i++, n++)
-            cacheParking.put(i, new Parking(i, "Parking #" + n, n * 10));
-
-        IgniteCache<Integer, Car> cacheCar = ignite.cache(CAR_CACHE_NAME);
-
-        for (int i = 0, n = 1; i < CAR_CNT; i++, n++)
-            cacheCar.put(i, new Car(i, rnd.nextInt(PARK_CNT), "Car #" + n));
-
-        if (log.isDebugEnabled())
-            log.debug("DEMO: Finished cars population.");
-    }
-
-    /**
-     * Creates a thread pool that can schedule commands to run after a given delay, or to execute periodically.
-     *
-     * @param corePoolSize Number of threads to keep in the pool, even if they are idle.
-     * @param threadName Part of thread name that would be used by thread factory.
-     * @return Newly created scheduled thread pool.
-     */
-    private static ScheduledExecutorService newScheduledThreadPool(int corePoolSize, final String threadName) {
-        ScheduledExecutorService srvc = Executors.newScheduledThreadPool(corePoolSize, new ThreadFactory() {
-            @Override public Thread newThread(Runnable r) {
-                Thread thread = new Thread(r, String.format("%s-%d", threadName, THREAD_CNT.getAndIncrement()));
-
-                thread.setDaemon(true);
-
-                return thread;
-            }
-        });
-
-        ScheduledThreadPoolExecutor executor = (ScheduledThreadPoolExecutor)srvc;
-
-        // Setting up shutdown policy.
-        executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
-        executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
-
-        return srvc;
-    }
-
-    /**
      * Starts read and write from cache in background.
      *
      * @param ignite Ignite.
      * @param cnt - maximum count read/write key
      */
     private static void startLoad(final Ignite ignite, final int cnt) {
-        final long diff = new java.util.Date().getTime();
-
-        populateCacheEmployee(ignite, diff);
-        populateCacheCar(ignite);
-
-        ScheduledExecutorService cachePool = newScheduledThreadPool(2, "demo-sql-load-cache-tasks");
-
-        cachePool.scheduleWithFixedDelay(new Runnable() {
-            @Override public void run() {
-                try {
-                    for (String cacheName : ignite.cacheNames()) {
-                        if (!DEMO_CACHES.contains(cacheName)) {
-                            IgniteCache<Integer, String> otherCache = ignite.cache(cacheName);
-
-                            if (otherCache != null) {
-                                for (int i = 0, n = 1; i < cnt; i++, n++) {
-                                    Integer key = rnd.nextInt(1000);
-
-                                    String val = otherCache.get(key);
-
-                                    if (val == null)
-                                        otherCache.put(key, "other-" + key);
-                                    else if (rnd.nextInt(100) < 30)
-                                        otherCache.remove(key);
-                                }
-                            }
-                        }
-                    }
-
-                    IgniteCache<Integer, Employee> cacheEmployee = ignite.cache(EMPLOYEE_CACHE_NAME);
-
-                    if (cacheEmployee != null)
-                        try(Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                            for (int i = 0, n = 1; i < cnt; i++, n++) {
-                                Integer id = rnd.nextInt(EMPL_CNT);
-
-                                Integer depId = rnd.nextInt(DEP_CNT);
-
-                                double r = rnd.nextDouble();
-
-                                cacheEmployee.put(id, new Employee(id, depId, depId, "First name employee #" + n,
-                                    "Last name employee #" + n, "Email employee #" + n, "Phone number employee #" + n,
-                                    new java.sql.Date((long)(r * diff)), "Job employee #" + n, 500 + round(r * 2000, 2)));
-
-                                if (rnd.nextBoolean())
-                                    cacheEmployee.remove(rnd.nextInt(EMPL_CNT));
-
-                                cacheEmployee.get(rnd.nextInt(EMPL_CNT));
-                            }
-
-                            if (rnd.nextInt(100) > 20)
-                                tx.commit();
-                        }
-                }
-                catch (Throwable e) {
-                    if (!e.getMessage().contains("cache is stopped"))
-                        ignite.log().error("Cache write task execution error", e);
-                }
-            }
-        }, 10, 3, TimeUnit.SECONDS);
-
-        cachePool.scheduleWithFixedDelay(new Runnable() {
-            @Override public void run() {
-                try {
-                    IgniteCache<Integer, Car> cache = ignite.cache(CAR_CACHE_NAME);
-
-                    if (cache != null)
-                        for (int i = 0; i < cnt; i++) {
-                            Integer carId = rnd.nextInt(CAR_CNT);
-
-                            cache.put(carId, new Car(carId, rnd.nextInt(PARK_CNT), "Car #" + (i + 1)));
-
-                            if (rnd.nextBoolean())
-                                cache.remove(rnd.nextInt(CAR_CNT));
-                        }
-                }
-                catch (IllegalStateException ignored) {
-                    // No-op.
-                }
-                catch (Throwable e) {
-                    if (!e.getMessage().contains("cache is stopped"))
-                        ignite.log().error("Cache write task execution error", e);
-                }
-            }
-        }, 10, 3, TimeUnit.SECONDS);
+        ignite.services().deployClusterSingleton("Demo caches load service", new DemoCachesLoadService(cnt));
+        ignite.services().deployNodeSingleton("RandomCache load service", new DemoRandomCacheLoadService(cnt));
     }
 
     /**
@@ -609,6 +160,14 @@ public class AgentClusterDemo {
                     }
                 }, 10, 10, TimeUnit.SECONDS);
 
+                IgniteServices services = ignite.services();
+
+                services.deployMultiple("Demo service: Multiple instances", new DemoServiceMultipleInstances(), 7, 3);
+                services.deployNodeSingleton("Demo service: Node singleton", new DemoServiceNodeSingleton());
+                services.deployClusterSingleton("Demo service: Cluster singleton", new DemoServiceClusterSingleton());
+                services.deployKeyAffinitySingleton("Demo service: Key affinity singleton",
+                    new DemoServiceKeyAffinity(), DemoCachesLoadService.CAR_CACHE_NAME, "id");
+
                 if (log.isDebugEnabled())
                     log.debug("DEMO: Started embedded nodes with indexed enabled caches...");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentDemoUtils.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentDemoUtils.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentDemoUtils.java
new file mode 100644
index 0000000..fb34de7
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentDemoUtils.java
@@ -0,0 +1,79 @@
+/*
+ * 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.console.demo;
+
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Utilites for Agent demo mode.
+ */
+public class AgentDemoUtils {
+    /** Counter for threads in pool. */
+    private static final AtomicInteger THREAD_CNT = new AtomicInteger(0);
+
+    /**
+     * Creates a thread pool that can schedule commands to run after a given delay, or to execute periodically.
+     *
+     * @param corePoolSize Number of threads to keep in the pool, even if they are idle.
+     * @param threadName Part of thread name that would be used by thread factory.
+     * @return Newly created scheduled thread pool.
+     */
+    public static ScheduledExecutorService newScheduledThreadPool(int corePoolSize, final String threadName) {
+        ScheduledExecutorService srvc = Executors.newScheduledThreadPool(corePoolSize, new ThreadFactory() {
+            @Override public Thread newThread(Runnable r) {
+                Thread thread = new Thread(r, String.format("%s-%d", threadName, THREAD_CNT.getAndIncrement()));
+
+                thread.setDaemon(true);
+
+                return thread;
+            }
+        });
+
+        ScheduledThreadPoolExecutor executor = (ScheduledThreadPoolExecutor)srvc;
+
+        // Setting up shutdown policy.
+        executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
+        executor.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
+
+        return srvc;
+    }
+
+    /**
+     * Round value.
+     *
+     * @param val Value to round.
+     * @param places Numbers after point.
+     * @return Rounded value;
+     */
+    public static double round(double val, int places) {
+        if (places < 0)
+            throw new IllegalArgumentException();
+
+        long factor = (long)Math.pow(10, places);
+
+        val *= factor;
+
+        long tmp = Math.round(val);
+
+        return (double)tmp / factor;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java
new file mode 100644
index 0000000..9117646
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoCachesLoadService.java
@@ -0,0 +1,456 @@
+/*
+ * 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.console.demo.service;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.console.demo.AgentDemoUtils;
+import org.apache.ignite.console.demo.model.Car;
+import org.apache.ignite.console.demo.model.Country;
+import org.apache.ignite.console.demo.model.Department;
+import org.apache.ignite.console.demo.model.Employee;
+import org.apache.ignite.console.demo.model.Parking;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+import org.apache.ignite.transactions.Transaction;
+
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Demo service. Create and populate caches. Run demo load on caches.
+ */
+public class DemoCachesLoadService implements Service {
+    /** Ignite instance. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Thread pool to execute cache load operations. */
+    private ScheduledExecutorService cachePool;
+
+    /** */
+    private static final String COUNTRY_CACHE_NAME = "CountryCache";
+
+    /** */
+    private static final String DEPARTMENT_CACHE_NAME = "DepartmentCache";
+
+    /** */
+    private static final String EMPLOYEE_CACHE_NAME = "EmployeeCache";
+
+    /** */
+    private static final String PARKING_CACHE_NAME = "ParkingCache";
+
+    /** */
+    public static final String CAR_CACHE_NAME = "CarCache";
+
+    /** */
+    static final Set<String> DEMO_CACHES = new HashSet<>(Arrays.asList(COUNTRY_CACHE_NAME,
+        DEPARTMENT_CACHE_NAME, EMPLOYEE_CACHE_NAME, PARKING_CACHE_NAME, CAR_CACHE_NAME));
+
+    /** Countries count. */
+    private static final int CNTR_CNT = 10;
+
+    /** Departments count */
+    private static final int DEP_CNT = 100;
+
+    /** Employees count. */
+    private static final int EMPL_CNT = 1000;
+
+    /** Countries count. */
+    private static final int CAR_CNT = 100;
+
+    /** Departments count */
+    private static final int PARK_CNT = 10;
+
+    /** */
+    private static final Random rnd = new Random();
+
+    /** Maximum count read/write key. */
+    private final int cnt;
+
+    /** Time range in milliseconds. */
+    private final long range;
+
+    /**
+     * @param cnt Maximum count read/write key.
+     */
+    public DemoCachesLoadService(int cnt) {
+        this.cnt = cnt;
+
+        range = new java.util.Date().getTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        if (cachePool != null)
+            cachePool.shutdown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        ignite.createCache(cacheCountry());
+        ignite.createCache(cacheDepartment());
+        ignite.createCache(cacheEmployee());
+        ignite.createCache(cacheCar());
+        ignite.createCache(cacheParking());
+
+        populateCacheEmployee();
+        populateCacheCar();
+
+        cachePool = AgentDemoUtils.newScheduledThreadPool(2, "demo-sql-load-cache-tasks");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        cachePool.scheduleWithFixedDelay(new Runnable() {
+            @Override public void run() {
+                try {
+                    IgniteCache<Integer, Employee> cacheEmployee = ignite.cache(EMPLOYEE_CACHE_NAME);
+
+                    if (cacheEmployee != null)
+                        try(Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                            for (int i = 0, n = 1; i < cnt; i++, n++) {
+                                Integer id = rnd.nextInt(EMPL_CNT);
+
+                                Integer depId = rnd.nextInt(DEP_CNT);
+
+                                double r = rnd.nextDouble();
+
+                                cacheEmployee.put(id, new Employee(id, depId, depId, "First name employee #" + n,
+                                    "Last name employee #" + n, "Email employee #" + n, "Phone number employee #" + n,
+                                    new java.sql.Date((long)(r * range)), "Job employee #" + n,
+                                    500 + AgentDemoUtils.round(r * 2000, 2)));
+
+                                if (rnd.nextBoolean())
+                                    cacheEmployee.remove(rnd.nextInt(EMPL_CNT));
+
+                                cacheEmployee.get(rnd.nextInt(EMPL_CNT));
+                            }
+
+                            if (rnd.nextInt(100) > 20)
+                                tx.commit();
+                        }
+                }
+                catch (Throwable e) {
+                    if (!e.getMessage().contains("cache is stopped"))
+                        ignite.log().error("Cache write task execution error", e);
+                }
+            }
+        }, 10, 3, TimeUnit.SECONDS);
+
+        cachePool.scheduleWithFixedDelay(new Runnable() {
+            @Override public void run() {
+                try {
+                    IgniteCache<Integer, Car> cache = ignite.cache(CAR_CACHE_NAME);
+
+                    if (cache != null)
+                        for (int i = 0; i < cnt; i++) {
+                            Integer carId = rnd.nextInt(CAR_CNT);
+
+                            cache.put(carId, new Car(carId, rnd.nextInt(PARK_CNT), "Car #" + (i + 1)));
+
+                            if (rnd.nextBoolean())
+                                cache.remove(rnd.nextInt(CAR_CNT));
+                        }
+                }
+                catch (IllegalStateException ignored) {
+                    // No-op.
+                }
+                catch (Throwable e) {
+                    if (!e.getMessage().contains("cache is stopped"))
+                        ignite.log().error("Cache write task execution error", e);
+                }
+            }
+        }, 10, 3, TimeUnit.SECONDS);
+    }
+
+
+    /**
+     * Create base cache configuration.
+     *
+     * @param name cache name.
+     * @return Cache configuration with basic properties set.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheConfiguration(String name) {
+        CacheConfiguration<K, V> ccfg = new CacheConfiguration<>(name);
+
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setQueryDetailMetricsSize(10);
+        ccfg.setStartSize(100);
+        ccfg.setStatisticsEnabled(true);
+
+        return ccfg;
+    }
+
+    /**
+     * Configure cacheCountry.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheCountry() {
+        CacheConfiguration<K, V> ccfg = cacheConfiguration(COUNTRY_CACHE_NAME);
+
+        // Configure cacheCountry types.
+        Collection<QueryEntity> qryEntities = new ArrayList<>();
+
+        // COUNTRY.
+        QueryEntity type = new QueryEntity();
+
+        qryEntities.add(type);
+
+        type.setKeyType(Integer.class.getName());
+        type.setValueType(Country.class.getName());
+
+        // Query fields for COUNTRY.
+        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
+
+        qryFlds.put("id", "java.lang.Integer");
+        qryFlds.put("name", "java.lang.String");
+        qryFlds.put("population", "java.lang.Integer");
+
+        type.setFields(qryFlds);
+
+        ccfg.setQueryEntities(qryEntities);
+
+        return ccfg;
+    }
+
+    /**
+     * Configure cacheEmployee.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheDepartment() {
+        CacheConfiguration<K, V> ccfg = cacheConfiguration(DEPARTMENT_CACHE_NAME);
+
+        // Configure cacheDepartment types.
+        Collection<QueryEntity> qryEntities = new ArrayList<>();
+
+        // DEPARTMENT.
+        QueryEntity type = new QueryEntity();
+
+        qryEntities.add(type);
+
+        type.setKeyType(Integer.class.getName());
+        type.setValueType(Department.class.getName());
+
+        // Query fields for DEPARTMENT.
+        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
+
+        qryFlds.put("id", "java.lang.Integer");
+        qryFlds.put("countryId", "java.lang.Integer");
+        qryFlds.put("name", "java.lang.String");
+
+        type.setFields(qryFlds);
+
+        ccfg.setQueryEntities(qryEntities);
+
+        return ccfg;
+    }
+
+    /**
+     * Configure cacheEmployee.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheEmployee() {
+        CacheConfiguration<K, V> ccfg = cacheConfiguration(EMPLOYEE_CACHE_NAME);
+
+        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        ccfg.setBackups(1);
+
+        // Configure cacheEmployee types.
+        Collection<QueryEntity> qryEntities = new ArrayList<>();
+
+        // EMPLOYEE.
+        QueryEntity type = new QueryEntity();
+
+        qryEntities.add(type);
+
+        type.setKeyType(Integer.class.getName());
+        type.setValueType(Employee.class.getName());
+
+        // Query fields for EMPLOYEE.
+        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
+
+        qryFlds.put("id", "java.lang.Integer");
+        qryFlds.put("departmentId", "java.lang.Integer");
+        qryFlds.put("managerId", "java.lang.Integer");
+        qryFlds.put("firstName", "java.lang.String");
+        qryFlds.put("lastName", "java.lang.String");
+        qryFlds.put("email", "java.lang.String");
+        qryFlds.put("phoneNumber", "java.lang.String");
+        qryFlds.put("hireDate", "java.sql.Date");
+        qryFlds.put("job", "java.lang.String");
+        qryFlds.put("salary", "java.lang.Double");
+
+        type.setFields(qryFlds);
+
+        // Indexes for EMPLOYEE.
+        Collection<QueryIndex> indexes = new ArrayList<>();
+
+        QueryIndex idx = new QueryIndex();
+
+        idx.setName("EMP_NAMES");
+        idx.setIndexType(QueryIndexType.SORTED);
+        LinkedHashMap<String, Boolean> indFlds = new LinkedHashMap<>();
+
+        indFlds.put("firstName", Boolean.FALSE);
+        indFlds.put("lastName", Boolean.FALSE);
+
+        idx.setFields(indFlds);
+
+        indexes.add(idx);
+        indexes.add(new QueryIndex("salary", QueryIndexType.SORTED, false, "EMP_SALARY"));
+
+        type.setIndexes(indexes);
+
+        ccfg.setQueryEntities(qryEntities);
+
+        return ccfg;
+    }
+
+    /**
+     * Configure cacheEmployee.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheParking() {
+        CacheConfiguration<K, V> ccfg = cacheConfiguration(PARKING_CACHE_NAME);
+
+        // Configure cacheParking types.
+        Collection<QueryEntity> qryEntities = new ArrayList<>();
+
+        // PARKING.
+        QueryEntity type = new QueryEntity();
+
+        qryEntities.add(type);
+
+        type.setKeyType(Integer.class.getName());
+        type.setValueType(Parking.class.getName());
+
+        // Query fields for PARKING.
+        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
+
+        qryFlds.put("id", "java.lang.Integer");
+        qryFlds.put("name", "java.lang.String");
+        qryFlds.put("capacity", "java.lang.Integer");
+
+        type.setFields(qryFlds);
+
+        ccfg.setQueryEntities(qryEntities);
+
+        return ccfg;
+    }
+
+    /**
+     * Configure cacheEmployee.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheCar() {
+        CacheConfiguration<K, V> ccfg = cacheConfiguration(CAR_CACHE_NAME);
+
+        // Configure cacheCar types.
+        Collection<QueryEntity> qryEntities = new ArrayList<>();
+
+        // CAR.
+        QueryEntity type = new QueryEntity();
+
+        qryEntities.add(type);
+
+        type.setKeyType(Integer.class.getName());
+        type.setValueType(Car.class.getName());
+
+        // Query fields for CAR.
+        LinkedHashMap<String, String> qryFlds = new LinkedHashMap<>();
+
+        qryFlds.put("id", "java.lang.Integer");
+        qryFlds.put("parkingId", "java.lang.Integer");
+        qryFlds.put("name", "java.lang.String");
+
+        type.setFields(qryFlds);
+
+        ccfg.setQueryEntities(qryEntities);
+
+        return ccfg;
+    }
+
+    /** */
+    private void populateCacheEmployee() {
+        if (ignite.log().isDebugEnabled())
+            ignite.log().debug("DEMO: Start employees population with data...");
+
+        IgniteCache<Integer, Country> cacheCountry = ignite.cache(COUNTRY_CACHE_NAME);
+
+        for (int i = 0, n = 1; i < CNTR_CNT; i++, n++)
+            cacheCountry.put(i, new Country(i, "Country #" + n, n * 10000000));
+
+        IgniteCache<Integer, Department> cacheDepartment = ignite.cache(DEPARTMENT_CACHE_NAME);
+
+        IgniteCache<Integer, Employee> cacheEmployee = ignite.cache(EMPLOYEE_CACHE_NAME);
+
+        for (int i = 0, n = 1; i < DEP_CNT; i++, n++) {
+            cacheDepartment.put(i, new Department(n, rnd.nextInt(CNTR_CNT), "Department #" + n));
+
+            double r = rnd.nextDouble();
+
+            cacheEmployee.put(i, new Employee(i, rnd.nextInt(DEP_CNT), null, "First name manager #" + n,
+                "Last name manager #" + n, "Email manager #" + n, "Phone number manager #" + n,
+                new java.sql.Date((long)(r * range)), "Job manager #" + n, 1000 + AgentDemoUtils.round(r * 4000, 2)));
+        }
+
+        for (int i = 0, n = 1; i < EMPL_CNT; i++, n++) {
+            Integer depId = rnd.nextInt(DEP_CNT);
+
+            double r = rnd.nextDouble();
+
+            cacheEmployee.put(i, new Employee(i, depId, depId, "First name employee #" + n,
+                "Last name employee #" + n, "Email employee #" + n, "Phone number employee #" + n,
+                new java.sql.Date((long)(r * range)), "Job employee #" + n, 500 + AgentDemoUtils.round(r * 2000, 2)));
+        }
+
+        if (ignite.log().isDebugEnabled())
+            ignite.log().debug("DEMO: Finished employees population.");
+    }
+
+    /** */
+    private void populateCacheCar() {
+        if (ignite.log().isDebugEnabled())
+            ignite.log().debug("DEMO: Start cars population...");
+
+        IgniteCache<Integer, Parking> cacheParking = ignite.cache(PARKING_CACHE_NAME);
+
+        for (int i = 0, n = 1; i < PARK_CNT; i++, n++)
+            cacheParking.put(i, new Parking(i, "Parking #" + n, n * 10));
+
+        IgniteCache<Integer, Car> cacheCar = ignite.cache(CAR_CACHE_NAME);
+
+        for (int i = 0, n = 1; i < CAR_CNT; i++, n++)
+            cacheCar.put(i, new Car(i, rnd.nextInt(PARK_CNT), "Car #" + n));
+
+        if (ignite.log().isDebugEnabled())
+            ignite.log().debug("DEMO: Finished cars population.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
new file mode 100644
index 0000000..57b26a2
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoRandomCacheLoadService.java
@@ -0,0 +1,120 @@
+/*
+ * 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.console.demo.service;
+
+import java.util.Random;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.console.demo.AgentDemoUtils;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+
+/**
+ * Demo service. Create cache and populate it by random int pairs.
+ */
+public class DemoRandomCacheLoadService implements Service {
+    /** Ignite instance. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Thread pool to execute cache load operations. */
+    private ScheduledExecutorService cachePool;
+
+    /** */
+    public static final String RANDOM_CACHE_NAME = "RandomCache";
+
+    /** Employees count. */
+    private static final int RND_CNT = 1024;
+
+    /** */
+    private static final Random rnd = new Random();
+
+    /** Maximum count read/write key. */
+    private final int cnt;
+
+    /**
+     * @param cnt Maximum count read/write key.
+     */
+    public DemoRandomCacheLoadService(int cnt) {
+        this.cnt = cnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        if (cachePool != null)
+            cachePool.shutdown();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        ignite.getOrCreateCache(cacheRandom());
+
+        cachePool = AgentDemoUtils.newScheduledThreadPool(2, "demo-sql-random-load-cache-tasks");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        cachePool.scheduleWithFixedDelay(new Runnable() {
+            @Override public void run() {
+                try {
+                    for (String cacheName : ignite.cacheNames()) {
+                        if (!DemoCachesLoadService.DEMO_CACHES.contains(cacheName)) {
+                            IgniteCache<Integer, Integer> cache = ignite.cache(cacheName);
+
+                            if (cache != null) {
+                                for (int i = 0, n = 1; i < cnt; i++, n++) {
+                                    Integer key = rnd.nextInt(RND_CNT);
+                                    Integer val = rnd.nextInt(RND_CNT);
+
+                                    cache.put(key, val);
+
+                                    if (rnd.nextInt(100) < 30)
+                                        cache.remove(key);
+                                }
+                            }
+                        }
+                    }
+                }
+                catch (Throwable e) {
+                    if (!e.getMessage().contains("cache is stopped"))
+                        ignite.log().error("Cache write task execution error", e);
+                }
+            }
+        }, 10, 3, TimeUnit.SECONDS);
+    }
+
+    /**
+     * Configure cacheCountry.
+     */
+    private static <K, V> CacheConfiguration<K, V> cacheRandom() {
+        CacheConfiguration<K, V> ccfg = new CacheConfiguration<>(RANDOM_CACHE_NAME);
+
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setQueryDetailMetricsSize(10);
+        ccfg.setStartSize(100);
+        ccfg.setStatisticsEnabled(true);
+        ccfg.setIndexedTypes(Integer.class, Integer.class);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceClusterSingleton.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceClusterSingleton.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceClusterSingleton.java
new file mode 100644
index 0000000..8c0623a
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceClusterSingleton.java
@@ -0,0 +1,41 @@
+/*
+ * 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.console.demo.service;
+
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+
+/**
+ * Demo service to provide on one node in cluster.
+ */
+public class DemoServiceClusterSingleton implements Service {
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceKeyAffinity.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceKeyAffinity.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceKeyAffinity.java
new file mode 100644
index 0000000..081ae27
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceKeyAffinity.java
@@ -0,0 +1,41 @@
+/*
+ * 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.console.demo.service;
+
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+
+/**
+ * Demo service to provide for cache.
+ */
+public class DemoServiceKeyAffinity implements Service {
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceMultipleInstances.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceMultipleInstances.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceMultipleInstances.java
new file mode 100644
index 0000000..0d10753
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceMultipleInstances.java
@@ -0,0 +1,41 @@
+/*
+ * 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.console.demo.service;
+
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+
+/**
+ * Demo service to provide on all nodes.
+ */
+public class DemoServiceMultipleInstances implements Service {
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a600cafd/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceNodeSingleton.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceNodeSingleton.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceNodeSingleton.java
new file mode 100644
index 0000000..4d491da
--- /dev/null
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/service/DemoServiceNodeSingleton.java
@@ -0,0 +1,41 @@
+/*
+ * 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.console.demo.service;
+
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+
+/**
+ * Demo service to provide on all nodes by one.
+ */
+public class DemoServiceNodeSingleton implements Service {
+    /** {@inheritDoc} */
+    @Override public void cancel(ServiceContext ctx) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void init(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(ServiceContext ctx) throws Exception {
+        // No-op.
+    }
+}