You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2018/11/06 09:41:34 UTC

ignite git commit: IGNITE-10101 Web Console: Refactored logic of collecting cache names for "Queries" screen.

Repository: ignite
Updated Branches:
  refs/heads/master 834bb1e7b -> 86b235104


IGNITE-10101 Web Console: Refactored logic of collecting cache names for "Queries" screen.


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

Branch: refs/heads/master
Commit: 86b2351048e182ed71de67336e5e51309f56ff25
Parents: 834bb1e
Author: Andrey Novikov <an...@apache.org>
Authored: Tue Nov 6 16:41:08 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Tue Nov 6 16:41:08 2018 +0700

----------------------------------------------------------------------
 .../web-console/backend/app/browsersHandler.js  |   5 +
 .../components/queries-notebook/controller.js   | 138 ++++++++-----------
 .../queries-notebook/template.tpl.pug           |   4 +-
 .../components/page-queries/notebook.data.js    |   3 -
 .../app/modules/agent/AgentManager.service.js   |  52 ++++++-
 5 files changed, 110 insertions(+), 92 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/86b23510/modules/web-console/backend/app/browsersHandler.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browsersHandler.js b/modules/web-console/backend/app/browsersHandler.js
index a18b467..a7156c5 100644
--- a/modules/web-console/backend/app/browsersHandler.js
+++ b/modules/web-console/backend/app/browsersHandler.js
@@ -244,6 +244,11 @@ module.exports = {
 
                 this.registerVisorTask('toggleClusterState', internalVisor('misc.VisorChangeGridActiveStateTask'), internalVisor('misc.VisorChangeGridActiveStateTaskArg'));
 
+                this.registerVisorTask('cacheNamesCollectorTask', internalVisor('cache.VisorCacheNamesCollectorTask'), 'java.lang.Void');
+
+                this.registerVisorTask('cacheNodesTask', internalVisor('cache.VisorCacheNodesTask'), 'java.lang.String');
+                this.registerVisorTask('cacheNodesTaskX2', internalVisor('cache.VisorCacheNodesTask'), internalVisor('cache.VisorCacheNodesTaskArg'));
+
                 // Return command result from grid to browser.
                 sock.on('node:visor', (arg, cb) => {
                     const {clusterId, params, credentials} = arg || {};

http://git-wip-us.apache.org/repos/asf/ignite/blob/86b23510/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js
index 3ee08b3..2cb328c 100644
--- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js
+++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/controller.js
@@ -14,6 +14,8 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
+import _ from 'lodash';
 import {nonEmpty, nonNil} from 'app/utils/lodashMixins';
 import id8 from 'app/utils/id8';
 import 'rxjs/add/operator/mergeMap';
@@ -891,37 +893,13 @@ export class NotebookCtrl {
         /**
          * Update caches list.
          */
-        const _refreshFn = () => {
-            return agentMgr.topology(true)
-                .then((nodes) => {
-                    $scope.caches = _.sortBy(_.reduce(nodes, (cachesAcc, node) => {
-                        _.forEach(node.caches, (cache) => {
-                            let item = _.find(cachesAcc, {name: cache.name});
-
-                            if (_.isNil(item)) {
-                                cache.label = maskCacheName(cache.name, true);
-                                cache.value = cache.name;
-
-                                cache.nodes = [];
-
-                                cachesAcc.push(item = cache);
-                            }
-
-                            item.nodes.push({
-                                nid: node.nodeId.toUpperCase(),
-                                ip: _.head(node.attributes['org.apache.ignite.ips'].split(', ')),
-                                version: node.attributes['org.apache.ignite.build.ver'],
-                                gridName: node.attributes['org.apache.ignite.ignite.name'],
-                                os: `${node.attributes['os.name']} ${node.attributes['os.arch']} ${node.attributes['os.version']}`,
-                                client: node.attributes['org.apache.ignite.cache.client']
-                            });
-                        });
-
-                        return cachesAcc;
-                    }, []), (cache) => cache.label.toLowerCase());
-
-                    // Reset to first cache in case of stopped selected.
-                    const cacheNames = _.map($scope.caches, (cache) => cache.value);
+        const _refreshCaches = () => {
+            return agentMgr.publicCacheNames()
+                .then((cacheNames) => {
+                    $scope.caches = _.sortBy(_.map(cacheNames, (name) => ({
+                        label: maskCacheName(name, true),
+                        value: name
+                    })), (cache) => cache.label.toLowerCase());
 
                     _.forEach($scope.notebook.paragraphs, (paragraph) => {
                         if (!_.includes(cacheNames, paragraph.cacheName))
@@ -936,6 +914,8 @@ export class NotebookCtrl {
 
                         _.forEach($scope.notebook.paragraphs, (paragraph) => $scope.execute(paragraph));
                     }
+
+                    $scope.$applyAsync();
                 })
                 .catch((err) => Messages.showError(err));
         };
@@ -950,7 +930,7 @@ export class NotebookCtrl {
             }).take(1);
 
             const refreshCaches = (period) => {
-                return timer(0, period).exhaustMap(() => _refreshFn()).merge(finishLoading$);
+                return timer(0, period).exhaustMap(() => _refreshCaches()).merge(finishLoading$);
             };
 
             this.refresh$ = awaitClusters$
@@ -1390,29 +1370,48 @@ export class NotebookCtrl {
 
         /**
          * @param {String} name Cache name.
-         * @return {Array.<String>} Nids
+         * @param {Array.<String>} nids Cache name.
+         * @return {Promise<Array.<{nid: string, ip: string, version:string, gridName: string, os: string, client: boolean}>>}
          */
-        const cacheNodes = (name) => {
-            return _.find($scope.caches, {name}).nodes;
+        const cacheNodesModel = (name, nids) => {
+            return agentMgr.topology(true)
+                .then((nodes) =>
+                    _.reduce(nodes, (acc, node) => {
+                        if (_.includes(nids, node.nodeId)) {
+                            acc.push({
+                                nid: node.nodeId.toUpperCase(),
+                                ip: _.head(node.attributes['org.apache.ignite.ips'].split(', ')),
+                                version: node.attributes['org.apache.ignite.build.ver'],
+                                gridName: node.attributes['org.apache.ignite.ignite.name'],
+                                os: `${node.attributes['os.name']} ${node.attributes['os.arch']} ${node.attributes['os.version']}`,
+                                client: node.attributes['org.apache.ignite.cache.client']
+                            });
+                        }
+
+                        return acc;
+                    }, [])
+                );
         };
 
         /**
-         * @param {String} name Cache name.
-         * @param {Boolean} local Local query.
-         * @return {String} Nid
+         * @param {string} name Cache name.
+         * @param {boolean} local Local query.
+         * @return {Promise<string>} Nid
          */
         const _chooseNode = (name, local) => {
             if (_.isEmpty(name))
                 return Promise.resolve(null);
 
-            const nodes = _.filter(cacheNodes(name), (node) => !node.client);
-
-            if (local) {
-                return Nodes.selectNode(nodes, name)
-                    .then((selectedNids) => _.head(selectedNids));
-            }
+            return agentMgr.cacheNodes(name)
+                .then((nids) => {
+                    if (local) {
+                        return cacheNodesModel(name, nids)
+                            .then((nodes) => Nodes.selectNode(nodes, name).catch(() => {}))
+                            .then((selectedNids) => _.head(selectedNids));
+                    }
 
-            return Promise.resolve(nodes[_.random(0, nodes.length - 1)].nid);
+                    return nids[_.random(0, nids.length - 1)];
+                });
         };
 
         const _executeRefresh = (paragraph) => {
@@ -1446,53 +1445,28 @@ export class NotebookCtrl {
             ${query} 
             ) LIMIT ${limitSize}`;
 
-        $scope.nonCollocatedJoinsAvailable = (paragraph) => {
-            const cache = _.find($scope.caches, {name: paragraph.cacheName});
-
-            if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, NON_COLLOCATED_JOINS_SINCE));
-
-            return false;
+        $scope.nonCollocatedJoinsAvailable = () => {
+            return Version.since(this.agentMgr.clusterVersion, NON_COLLOCATED_JOINS_SINCE);
         };
 
-        $scope.collocatedJoinsAvailable = (paragraph) => {
-            const cache = _.find($scope.caches, {name: paragraph.cacheName});
-
-            if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...COLLOCATED_QUERY_SINCE));
-
-            return false;
+        $scope.collocatedJoinsAvailable = () => {
+            return Version.since(this.agentMgr.clusterVersion, ...COLLOCATED_QUERY_SINCE);
         };
 
-        $scope.enforceJoinOrderAvailable = (paragraph) => {
-            const cache = _.find($scope.caches, {name: paragraph.cacheName});
-
-            if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...ENFORCE_JOIN_SINCE));
-
-            return false;
+        $scope.enforceJoinOrderAvailable = () => {
+            return Version.since(this.agentMgr.clusterVersion, ...ENFORCE_JOIN_SINCE);
         };
 
-        $scope.lazyQueryAvailable = (paragraph) => {
-            const cache = _.find($scope.caches, {name: paragraph.cacheName});
-
-            if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...LAZY_QUERY_SINCE));
-
-            return false;
+        $scope.lazyQueryAvailable = () => {
+            return Version.since(this.agentMgr.clusterVersion, ...LAZY_QUERY_SINCE);
         };
 
-        $scope.ddlAvailable = (paragraph) => {
-            const cache = _.find($scope.caches, {name: paragraph.cacheName});
-
-            if (cache)
-                return !!_.find(cache.nodes, (node) => Version.since(node.version, ...DDL_SINCE));
-
-            return false;
+        $scope.ddlAvailable = () => {
+            return Version.since(this.agentMgr.clusterVersion, ...DDL_SINCE);
         };
 
         $scope.cacheNameForSql = (paragraph) => {
-            return $scope.ddlAvailable(paragraph) && !paragraph.useAsDefaultSchema ? null : paragraph.cacheName;
+            return $scope.ddlAvailable() && !paragraph.useAsDefaultSchema ? null : paragraph.cacheName;
         };
 
         $scope.execute = (paragraph, local = false) => {
@@ -1893,7 +1867,7 @@ export class NotebookCtrl {
             agentMgr.metadata()
                 .then((metadata) => {
                     $scope.metadata = _.sortBy(_.filter(metadata, (meta) => {
-                        const cache = _.find($scope.caches, { name: meta.cacheName });
+                        const cache = _.find($scope.caches, { value: meta.cacheName });
 
                         if (cache) {
                             meta.name = (cache.sqlSchema || '"' + meta.cacheName + '"') + '.' + meta.typeName;

http://git-wip-us.apache.org/repos/asf/ignite/blob/86b23510/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug
index 781ce51..af673a8 100644
--- a/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug
+++ b/modules/web-console/frontend/app/components/page-queries/components/queries-notebook/template.tpl.pug
@@ -394,12 +394,12 @@ mixin paragraph-query
                         input.form-control(type='text' st-search='label' placeholder='Filter caches...')
 
                     .queries-notebook-displayed-caches
-                        div(ng-repeat='cache in displayedCaches track by cache.name')
+                        div(ng-repeat='cache in displayedCaches track by cache.value')
                             +form-field__radio({
                                 label: '{{ cache.label }}',
                                 model: 'paragraph.cacheName',
                                 name: '"cache_" + [paragraph.id, $index].join("_")',
-                                value: 'cache.name'
+                                value: 'cache.value'
                             })
 
                     .settings-row

http://git-wip-us.apache.org/repos/asf/ignite/blob/86b23510/modules/web-console/frontend/app/components/page-queries/notebook.data.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-queries/notebook.data.js b/modules/web-console/frontend/app/components/page-queries/notebook.data.js
index 6c784bf..d9035c2 100644
--- a/modules/web-console/frontend/app/components/page-queries/notebook.data.js
+++ b/modules/web-console/frontend/app/components/page-queries/notebook.data.js
@@ -21,7 +21,6 @@ const DEMO_NOTEBOOK = {
     paragraphs: [
         {
             name: 'Query with refresh rate',
-            cacheName: 'CarCache',
             qryType: 'query',
             pageSize: 100,
             limit: 0,
@@ -39,7 +38,6 @@ const DEMO_NOTEBOOK = {
         },
         {
             name: 'Simple query',
-            cacheName: 'CarCache',
             qryType: 'query',
             pageSize: 100,
             limit: 0,
@@ -54,7 +52,6 @@ const DEMO_NOTEBOOK = {
         },
         {
             name: 'Query with aggregates',
-            cacheName: 'ParkingCache',
             qryType: 'query',
             pageSize: 100,
             limit: 0,

http://git-wip-us.apache.org/repos/asf/ignite/blob/86b23510/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
index 7226cff..c59d6a1 100644
--- a/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
+++ b/modules/web-console/frontend/app/modules/agent/AgentManager.service.js
@@ -43,6 +43,15 @@ const State = {
 const IGNITE_2_0 = '2.0.0';
 const LAZY_QUERY_SINCE = [['2.1.4-p1', '2.2.0'], '2.2.1'];
 const COLLOCATED_QUERY_SINCE = [['2.3.5', '2.4.0'], ['2.4.6', '2.5.0'], ['2.5.1-p13', '2.6.0'], '2.7.0'];
+const COLLECT_BY_CACHE_GROUPS_SINCE = '2.7.0';
+
+/** Reserved cache names */
+const RESERVED_CACHE_NAMES = [
+    'ignite-hadoop-mr-sys-cache',
+    'ignite-sys-cache',
+    'MetaStorage',
+    'TxLog'
+];
 
 /** Error codes from o.a.i.internal.processors.restGridRestResponse.java */
 const SuccessStatus = {
@@ -125,7 +134,7 @@ export default class AgentManager {
     ClusterLoginSrv;
 
     /** @type {String} */
-    clusterVersion = '2.4.0';
+    clusterVersion;
 
     connectionSbj = new BehaviorSubject(new ConnectionState(AgentManager.restoreActiveCluster()));
 
@@ -171,6 +180,8 @@ export default class AgentManager {
         this.Version = Version;
         this.ClusterLoginSrv = ClusterLoginSrv;
 
+        this.clusterVersion = this.Version.webConsole;
+
         let prevCluster;
 
         this.currentCluster$ = this.connectionSbj
@@ -537,12 +548,43 @@ export default class AgentManager {
     }
 
     /**
-     * @param {Boolean} [attr]
-     * @param {Boolean} [mtr]
+     * @param {boolean} [attr] Collect node attributes.
+     * @param {boolean} [mtr] Collect node metrics.
+     * @param {boolean} [caches] Collect node caches descriptors.
      * @returns {Promise}
      */
-    topology(attr = false, mtr = false) {
-        return this._executeOnCluster('node:rest', {cmd: 'top', attr, mtr, caches: false});
+    topology(attr = false, mtr = false, caches = false) {
+        return this._executeOnCluster('node:rest', {cmd: 'top', attr, mtr, caches});
+    }
+
+    collectCacheNames(nid) {
+        if (this.available(COLLECT_BY_CACHE_GROUPS_SINCE))
+            return this.visorTask('cacheNamesCollectorTask', nid);
+
+        return Promise.resolve({cacheGroupsNotAvailable: true});
+    }
+
+    publicCacheNames() {
+        return this.collectCacheNames()
+            .then((data) => {
+                if (nonEmpty(data.caches))
+                    return _.difference(_.keys(data.caches), RESERVED_CACHE_NAMES);
+
+                return this.topology(false, false, true)
+                    .then((nodes) => {
+                        return _.map(_.uniqBy(_.flatMap(nodes, 'caches'), 'name'), 'name');
+                    });
+            });
+    }
+
+    /**
+     * @param {string} cacheName Cache name.
+     */
+    cacheNodes(cacheName) {
+        if (this.available(IGNITE_2_0))
+            return this.visorTask('cacheNodesTaskX2', null, cacheName);
+
+        return this.visorTask('cacheNodesTask', null, cacheName);
     }
 
     /**